You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Guozhang Wang <wa...@gmail.com> on 2017/06/02 00:27:24 UTC

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

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 <je...@gmail.com>
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 like to get your comments.
>
>
> [1]
> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> Re+DISCUSS+KIP+149+Enabling+key+access+in+ValueTransformer+ValueMapper+
> and+ValueJoiner
> [2]
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 159%3A+Introducing+Rich+functions+to+Streams
>
>
> Cheers,
> Jeyhun
> --
> -Cheers
>
> Jeyhun
>



-- 
-- Guozhang

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Moved this KIP into status "inactive". Feel free to resume and any time.

-Matthias

On 12/18/17 4:09 PM, Matthias J. Sax wrote:
> I just want to point out that the basic idea is great and that we should
> apply optimizations like "filter first" and other. But we should NOT
> convolute this KIP with orthogonal improvements.
> 
> In fact, we have an umbrella JIRA for DSL optimization already:
> https://issues.apache.org/jira/browse/KAFKA-6034
> 
> @Jan: feel free to create sub-tasks for new optimization ideas and we
> can take it from there.
> 
> 
> -Matthias
> 
> 
> On 12/18/17 7:55 AM, Bill Bejeck wrote:
>> Jan,
>>
>> I apologize for the delayed response.
>>
>> my suggestion would be that instead of
>>>
>>> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>>>
>>> we build
>>>
>>> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
>>
>>
>> I agree that filtering before the KTable source makes sense and would be a
>> positive change to implement.
>>
>> But the situation above is just one scenario out of many we need to
>> consider.  I'm not sure we can cover all the implications from different
>> use cases ahead of time.
>>
>> So I'm inclined to agree with Guozhang that we come up with clear "rules"
>> (I use the word rules for lack of a better term) for RecordContext usage
>> and inheritance. That way going forward we can have distinct expectations
>> of different use cases.
>>
>> -Bill
>>
>> On Fri, Dec 15, 2017 at 3:57 PM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Regarding the record context inheritance: I agree it may be a better idea
>>> for now to drop the information when we cannot come up with a consensus
>>> about how the record context should be inherited. Like Bill I was a bit
>>> worried about the lacking of such data lineage information for trouble
>>> shooting in operations or debugging in coding; but I think we can still try
>>> to come up with better solutions in the future by extending the current
>>> protocol, than coming up with something that we realized that we need to
>>> change in the future.
>>>
>>> Regarding the table / filter question: I agree with Jan that we could
>>> consider update the builder so that we will push down the filter earlier
>>> than KTable source that materialized the store; on the other hand, I think
>>> Matthias' point is that even doing this does not completely exclude the
>>> scenarios that you'd have the old/new pairs in your Tables, for example,
>>> consider:
>>>
>>> table1 = stream1.groupBy(...).aggregate(...)
>>> table2 = table1.filter(..., Materialized.as(...))
>>>
>>> In this case table2 is filtering on table1 which is not read from the
>>> source, and hence it already outputs the old/new pairs already, so we still
>>> need to consider how to handle it.
>>>
>>>
>>> So I'd suggest the following execution plan towards KIP-159:
>>>
>>> 1) revisit our record context (topic, partition, offset, timestamp)
>>> protocols that is used at the DSL layer, make it clear at which high-level
>>> operators we should apply certain inheritance rule, and which others we
>>> should drop such information.
>>>     1.1) modify the lower-level PAPI that DSL leverages, to allow the
>>> caller (DSL) to modify the record context (note that today for lower-level
>>> API, the record context is always passed through when forwarding to the
>>> next processor node)
>>> 2) at the same time, consider optimizing the source KTable filter cases (I
>>> think we already have certain JIRA tickets for this) so that the filter
>>> operator is pushed early than the KTABLESOURCE node where materialization
>>> happens.
>>> 3) after 1) is done, come back to KIP-159 and add the proposed APIs.
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Thu, Dec 7, 2017 at 12:27 PM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> Thank you Bill,
>>>>
>>>> I think this is reasonable. Do you have any suggestion
>>>> for handling oldValues in cases like
>>>>
>>>> builder.table().filter(RichPredicate).join()
>>>>
>>>> where we process a Change with old and new value and dont have a record
>>>> context for old.
>>>>
>>>> my suggestion would be that instead of
>>>>
>>>> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>>>>
>>>> we build
>>>>
>>>> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
>>>>
>>>> We should build a topology like this from the beginning and not have
>>>> an optimisation phase afterwards.
>>>>
>>>> Any opinions?
>>>>
>>>> Best Jan
>>>>
>>>>
>>>>
>>>>
>>>> On 05.12.2017 17:34, Bill Bejeck wrote:
>>>>
>>>>> Matthias,
>>>>>
>>>>> Overall I agree with what you've presented here.
>>>>>
>>>>> Initially, I was hesitant to remove information from the context of the
>>>>> result records (Joins or Aggregations) with the thought that when there
>>>>> are
>>>>> unexpected results, the source information would be useful for tracing
>>>>> back
>>>>> where the error could have occurred.  But in the case of Joins and
>>>>> Aggregations, the amount of data needed to do meaningful analysis could
>>> be
>>>>> too much. For example, a join result could come from two topics so you'd
>>>>> need to keep both original topic names, offsets, etc. (plus the broker
>>>>> could have deleted the records in the interim so even having offset
>>> could
>>>>> provide nothing).
>>>>>
>>>>> I'm bit long winded here, but I've come full circle to your original
>>>>> proposal that since Joins and Aggregations produce fundamentally new
>>>>> types,
>>>>> we drop the corresponding information from the context even in the case
>>> of
>>>>> single topic aggregations.
>>>>>
>>>>> Thanks,
>>>>> Bill
>>>>>
>>>>> On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>
>>>>> I agree with Guozhang that just exposing meta data at the source level
>>>>>> might not provide too much value. Furthermore, for timestamps we do
>>>>>> already have a well defined contract and we should exploit it:
>>>>>> timestamps can always be provided in a meaningful way.
>>>>>>
>>>>>> Also, for simple operations like KStream-filter/map the contract is
>>>>>> simple and we can just use it. Same for KTable-filter/map (for new
>>>>>> values).
>>>>>>
>>>>>> For aggregations, join, and oldValue, I could just drop some
>>> information
>>>>>> and return `null`/-1, if the result records has no semantically
>>>>>> meaningful meta data.
>>>>>>
>>>>>> For example, for aggregations, we could preserve the partition (as all
>>>>>> agg-input-records have the same partition). For single input topic
>>>>>> aggregation (what I guess is the most prominent case), we can also
>>> carry
>>>>>> over the topic name (would be a internal repartitioning topic name
>>>>>> often). Offsets don't have any semantic interpretation IMHO and we
>>> could
>>>>>> return -1.
>>>>>>
>>>>>> For joins, we could keep the partition information. Topic and offset
>>> are
>>>>>> both unknown/invalid for the output record IMHO.
>>>>>>
>>>>>> For the oldValue case, we can keep partition and for single input topic
>>>>>> case topic name. Timestamp might be -1 for now, but after we added
>>>>>> timestamps to KTable (what we plan to do anyway), we can also return a
>>>>>> valid timestamp. Offset would be -1 again (if we store offset in KTable
>>>>>> too, we could provide all offset as well -- but I don't see too much
>>>>>> value in doing this compared to the storage overhead this implies).
>>>>>>
>>>>>>
>>>>>> WDYT?
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 11/29/17 4:14 AM, Jan Filipiak wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> thank you for the summary and thanks for acknowledging that I do have
>>> a
>>>>>>> point here.
>>>>>>>
>>>>>>> I don't like the second Idea at all. Hence I started of this
>>> discussion.
>>>>>>>
>>>>>>> I am just disappointed, back then when we had the discussion about how
>>>>>>> to refactor store overload
>>>>>>> and IQ handling, I knew the path we are taking is wrong. Having
>>> problems
>>>>>>> implementing these kinda
>>>>>>> features (wich are really simple)  is just a symptom of messed up IQ
>>>>>>> implementation. I wish really bad
>>>>>>> I could have convinced you guys back then. To be honest with IQ we can
>>>>>>> continue here
>>>>>>> as we Materialize but would not send oldValue, but with join you're
>>> out
>>>>>>> of luck with current setup.
>>>>>>>
>>>>>>> I of course recommend to do not introduce any optimizations here. Id
>>>>>>> recommend to go towards what
>>>>>>> I recommended already back then. So i would't say we need to optimize
>>>>>>> anything later we need to build
>>>>>>> the topology better in the first place.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 28.11.2017 21:00, Guozhang Wang wrote:
>>>>>>>
>>>>>>>> Jan,
>>>>>>>>
>>>>>>>> Thanks for your input, I can understand now that the oldValue is also
>>>>>>>> exposed in user customized `filter` function and hence want record
>>>>>>>> context
>>>>>>>> we should expose is a problem. And I think it does brings a good
>>> point
>>>>>>>>
>>>>>>> to
>>>>>>
>>>>>>> consider for KIP-159. The discussions maybe a bit confusing to reader
>>>>>>>> though, and hence I'd like to summarize the status quo and with a
>>>>>>>> proposal:
>>>>>>>>
>>>>>>>> In today's Streams DSL, when a KTable is created either from a source
>>>>>>>> topic, or from an stateful operator, we will materialize the KTable
>>>>>>>> with a
>>>>>>>> backing state store; on the other hand, KTables created from a
>>>>>>>> non-stateful
>>>>>>>> operator like filter, will not be backed by a state store by default
>>>>>>>> unless
>>>>>>>> users indicate so (e.g. using the overloaded function with the
>>>>>>>> queryable
>>>>>>>> name or store supplier).
>>>>>>>>
>>>>>>>> For example:
>>>>>>>>
>>>>>>>> KTable table1 = builder.table("topic");
>>>>>>>>
>>>>>>> // a
>>>>>>
>>>>>>> state store created for table1
>>>>>>>> KTable table2 = table1.filter(..);
>>>>>>>> // no state store created for table2
>>>>>>>> KTable table3 = table1.filter(.., "storeName");                  // a
>>>>>>>> state
>>>>>>>> store created for table3
>>>>>>>> KTable table4 = table1.groupBy(..).aggregate(..);            // a
>>>>>>>> state
>>>>>>>> store created for table4
>>>>>>>>
>>>>>>>> Because of that, the filter() operator above on table1 will always be
>>>>>>>> exposed with oldValue and newValue; Damian's point is that, we may
>>>>>>>> optimize
>>>>>>>> the first case such that table1 will only be materialized if users
>>>>>>>> asked so
>>>>>>>> (e.g. using the overloaded function with a store supplier), and in
>>>>>>>> which
>>>>>>>> case, we do not need to pass newValue / oldValue pairs (I think this
>>> is
>>>>>>>> what Jan suggests as well, i.e. do filtering before materializing, so
>>>>>>>> that
>>>>>>>> we can have a smaller backed state store as well). But this
>>>>>>>> optimization
>>>>>>>> does not eliminate the possibilities that we may still need to do
>>>>>>>> filter if
>>>>>>>> users does specify "yes I do want to the source KTable itself to be
>>>>>>>> materialized, please". So the concern about how to expose the record
>>>>>>>> context in such cases still persists.
>>>>>>>>
>>>>>>>>
>>>>>>>> With that, regarding to KIP-159 itself, here are my thoughts:
>>>>>>>>
>>>>>>>> 1) if we restrict the scope of exposing record context only to source
>>>>>>>> KTables / KStreams I felt the KIP itself does not bring much value
>>>>>>>> given
>>>>>>>> its required API change because only the SourceKStream can safely
>>>>>>>> maintain
>>>>>>>> its records context, and for SourceKTable if it is materialized, then
>>>>>>>> even
>>>>>>>> non-stateful operators like Join may still have a concern about
>>>>>>>> exposing
>>>>>>>> the record context.
>>>>>>>>
>>>>>>>> 2) an alternative idea is we provide the semantics on how record
>>>>>>>> context
>>>>>>>> would be inherited across the operators for KTable / KStream and
>>>>>>>> expose it
>>>>>>>> in all operators (similarly in PAPI we would expose a much simpler
>>>>>>>> contract), and make it as a public contract that Streams library will
>>>>>>>> guarantee moving forward even we optimize our topology builder; it
>>> may
>>>>>>>> not
>>>>>>>> align perfectly with the linear algebraic semantics but practically
>>>>>>>> applicable for most cases; if users semantics do not fit in the
>>>>>>>> provided
>>>>>>>> contract, then they may need to get this themselves (embed such
>>>>>>>> information
>>>>>>>> in the value payload, for example).
>>>>>>>>
>>>>>>>> If people do not like the second idea, I'd suggest we hold on
>>> pursuing
>>>>>>>> the
>>>>>>>> first direction since to me its beneficial scope is too limited
>>>>>>>> compared to
>>>>>>>> its cost.
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <
>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Cleary we show the oldValue to the user. We have to, because we
>>> filter
>>>>>>>>> after the store.
>>>>>>>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>>>>>>>>> ain/java/org/apache/kafka/streams/kstream/internals/
>>>>>>>>>
>>>>>>>> KTableFilter.java#L96
>>>>>>
>>>>>>>
>>>>>>>>> I cannot help you following this. It is really obvious and I am
>>>>>>>>> running
>>>>>>>>> out of tools for explaining.
>>>>>>>>>
>>>>>>>>> Thanks for understanding my point to put filter before. Not only
>>>>>>>>> would it
>>>>>>>>> make the store smaller. It would make this feature reasonably
>>>>>>>>> possible and
>>>>>>>>> the framework easier. Interestingly it would also help to move IQ
>>>>>>>>> into more
>>>>>>>>> reasonable directions. And it might help understand that we do not
>>>>>>>>> need any
>>>>>>>>> intermediate representation of the topology,
>>>>>>>>>
>>>>>>>>> KIP-182 I have no clue what everyone has with their "bytestores" so
>>>>>>>>> broken. But putting another store after doesn't help when the store
>>>>>>>>> before
>>>>>>>>> is the problem.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>>>>>>>>
>>>>>>>>>    From a DSL point of view, users only see the new value on a
>>>>>>>>>> KTable#filter anyway. So why should it be an issue that we use
>>>>>>>>>> <newValue,oldValue> pair under the hood?
>>>>>>>>>>
>>>>>>>>>> User sees newValue and gets corresponding RecordContext. I can't
>>> see
>>>>>>>>>> any
>>>>>>>>>> issue here?
>>>>>>>>>>
>>>>>>>>>> I cannot follow here:
>>>>>>>>>>
>>>>>>>>>> Even when we have a statefull operation last. We move it to the
>>> very
>>>>>>>>>>
>>>>>>>>>>> first processor (KtableSource)
>>>>>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>>>>>
>>>>>>>>>>>> With regard to `builder.table().filter()`:
>>>>>>>>>>
>>>>>>>>>> I see you point that it would be good to be able to apply the
>>>>>>>>>> filter()
>>>>>>>>>> first to reduce the stat store size of the table. But how is this
>>>>>>>>>> related to KIP-159?
>>>>>>>>>>
>>>>>>>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>>>>>>>>> putting a custom dummy store into the table and materialize the
>>>>>>>>>> filter
>>>>>>>>>> result afterwards? It's not a nice way to do, but seems to be
>>>>>>>>>>
>>>>>>>>> possible.
>>>>>>
>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>>>>>>>>
>>>>>>>>>> The comment is valid. It falls exactly into this topic, it has
>>>>>>>>>>>
>>>>>>>>>> exactly
>>>>>>
>>>>>>> todo with this!
>>>>>>>>>>> Even when we have a statefull operation last. We move it to the
>>> very
>>>>>>>>>>> first processor (KtableSource)
>>>>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>>>>
>>>>>>>>>>> Regarding the other Jiras you are referring to. They harm the
>>>>>>>>>>> project
>>>>>>>>>>> more than they do good!
>>>>>>>>>>> There is no need for this kind of optimizer and meta
>>> representation
>>>>>>>>>>> and
>>>>>>>>>>> what not. I hope they
>>>>>>>>>>> never get implemented.
>>>>>>>>>>>
>>>>>>>>>>> Best Jan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>>>>>>>>
>>>>>>>>>>> Jan, i think you comment with respect to filtering is valid,
>>> though
>>>>>>>>>>>> not for
>>>>>>>>>>>> this KIP. We have separate JIRAs for topology optimization of
>>>>>>>>>>>> which this
>>>>>>>>>>>> falls into.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Damian
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Jan,
>>>>>>>>>>>>
>>>>>>>>>>>>> Not sure I understand your argument that "we still going to
>>>>>>>>>>>>> present
>>>>>>>>>>>>> change.oldValue to the filter even though the record context()
>>> is
>>>>>>>>>>>>> for
>>>>>>>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
>>>>>>>>>>>>> If yes
>>>>>>>>>>>>> could you point to me which LOC are you concerning about?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> a remark of mine that got missed during migration:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> There is this problem that even though we have
>>>>>>>>>>>>>> source.table.filter.join
>>>>>>>>>>>>>> the state-fullness happens at the table step not a the join
>>>>>>>>>>>>>> step. In a
>>>>>>>>>>>>>> filter
>>>>>>>>>>>>>> we still going to present change.oldValue to the filter even
>>>>>>>>>>>>>>
>>>>>>>>>>>>> though
>>>>>>
>>>>>>> the
>>>>>>>>>>>>>> record context() is for change.newValue. I would go as far as
>>>>>>>>>>>>>> applying
>>>>>>>>>>>>>> the filter before the table processor. Not to just get KIP-159,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> but
>>>>>>
>>>>>>> because
>>>>>>>>>>>>>
>>>>>>>>>>>>> I think its a side effect of a non ideal topology layout. If i
>>> can
>>>>>>>>>>>>>> filter
>>>>>>>>>>>>>> 99% of my
>>>>>>>>>>>>>> records. my state could be way smaller. Also widely escalates
>>> the
>>>>>>>>>>>>>> context
>>>>>>>>>>>>>> of the KIP
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails
>>>>>>>>>>>>>> were
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>> to VOTE thread.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Copying Guozhang's last summary below. Thanks for this
>>> summary.
>>>>>>>>>>>>>>> Very
>>>>>>>>>>>>>>> comprehensive!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>>>>>>>>> context
>>>>>>>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to
>>> (2)
>>>>>>>>>>>>>>> carrying the context to more operators than just sources. It
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>> seems,
>>>>>>>>>>>>>>> that we all agree, that many-to-one operations void the
>>> context.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I still think, that just going with plain (2) is too
>>>>>>>>>>>>>>> restrictive --
>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Also note, that the two operators filter() and filterNot()
>>> don't
>>>>>>>>>>>>>>> modify
>>>>>>>>>>>>>>> the record and thus for both, it would be absolutely valid to
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> keep
>>>>>>
>>>>>>> the
>>>>>>>>>>>>>>> context.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I personally would keep the context for at least all
>>> one-to-one
>>>>>>>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> the
>>>>>>
>>>>>>> context further: at least the offset information is
>>>>>>>>>>>>>>> questionable for
>>>>>>>>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>>>>>>>>> inherited
>>>>>>>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>>>>>>>>> "partition". Thus, I think it's still valuable information we
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>> carry
>>>>>>>>>>>>>>> downstreams.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Jan: which approach are you referring to as "the approach that
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> table would be perfect"?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>>>>>>>>> exposing the
>>>>>>>>>>>>>>>> record context which has the issues that we have been
>>>>>>>>>>>>>>>> discussing
>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>> and its semantics is always referring to the "processing
>>>>>>>>>>>>>>>> record" at
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> different:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1) the record traversed the topology from source to sink, it
>>>>>>>>>>>>>>>> may be
>>>>>>>>>>>>>>>> transformed into new object or even generate multiple new
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> objects
>>>>>>
>>>>>>> (think:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> branch) along the traversal. And the record context is
>>> referring
>>>>>>>>>>>>>>
>>>>>>>>>>>>> to
>>>>>>
>>>>>>> this
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> processing record. Here the "lifetime" of the record lasts for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> entire
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> treated as
>>>>>>>>>>>>>>>> different transformed values of this record (this applies to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> join
>>>>>>
>>>>>>> and
>>>>>>>>>>>>>>>> aggregations as well).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2) the record being processed is wiped out in the first
>>>>>>>>>>>>>>>> operator
>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> source, and NEW records are forwarded to downstream
>>> operators.
>>>>>>>>>>>>>>>> I.e.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> record only lives between two adjacent operators, once it
>>>>>>>>>>>>>> reached the
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think in the past we have talked about Streams under both
>>>>>>>>>>>>>>>> context,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically
>>> more
>>>>>>>>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ends at
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> join operator as it is only be materialized, while stream
>>>>>>>>>>>>>>>> record's
>>>>>>>>>>>>>>>> traversal goes through the join operator to further down
>>> until
>>>>>>>>>>>>>>>> sinks).
>>>>>>>>>>>>>>>> However if we are going to interpret following 2) above then
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> even
>>>>>>
>>>>>>> for
>>>>>>>>>>>>>>>> non-stateful operators we would not inherit record context.
>>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>>> we're
>>>>>>>>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3) a record would traverse "through" one-to-one
>>> (non-stateful)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> operators,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> will "replicate" at one-to-many (non-stateful) operators
>>> (think:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "mapValues"
>>>>>>>>>>>>>>>>       ) and will "end" at many-to-one (stateful) operators
>>>>>>>>>>>>>>>> where NEW
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>> page before chatting more.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
>>>>>>>>>>>>>>>> designs
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> punctuate() was not deprecated.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I corrected.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I just re-read the KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed
>>> completely
>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Jeyhun,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> carefully if we should add this functionality to the DSL
>>> layer
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>>>>>>>>> conclusion is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> it would require revamping the public APIs quite a lot, and
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> it's
>>>>>>
>>>>>>> not
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> clear
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> if it is a good trade-off than asking users to call
>>> process()
>>>>>>>>>>>>>>>>> instead.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
>>>>>>>>>>>>>>>>>> <da...@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         - on-demand commit() feature
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
>>>>>>>>>>>>>>>>>>> simplified
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
>>>>>>>>>>>>>>>>>>>> agree that
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
>>>>>>>>>>>>>>>>>>>>> close the
>>>>>>>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> best
>>>>>>
>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> do.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
>>>>>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> that
>>>>>>
>>>>>>> flexibility
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains
>>> processor
>>>>>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context
>>>>>>>>>>>>>>>>>>>>>> (offset,
>>>>>>>>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want
>>> to
>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call
>>> from
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion
>>> of
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>>>>>>>> `commit()`
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
>>>>>>>>>>>>>>>>>>>>> elaborate a
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that
>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
>>>>>>>>>>>>>>>>>>>>> better. Why
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit?
>>> To
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> explicitly,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my
>>> understanding,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> is an helper object that provide access to record meta
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> data.
>>>>>>
>>>>>>> Requesting
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> specific
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
>>>>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> path.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> internal
>>>>>>
>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
>>> is
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
>>>>>>>>>>>>>>>>>>>>>> `RecordContext` is
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> interfaces.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>> wants to commit, then there should be some method
>>> inside
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
>>>>>>>>>>>>>>>>>>>>>>>> snippet in
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>          public void process(final K1 key, final V1
>>>>>>>>>>>>>>>>>>>>> value) {
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>              recordContext = new RecordContext()
>>>>>>>>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>>                  public void commit() {
>>>>>>>>>>>>>>>>>>>>>>>>                      context().commit();
>>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>>                  public long offset() {
>>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().offs
>>>>>>
>>>>>>> et();
>>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>>                  public long timestamp() {
>>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>>                  public String topic() {
>>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().topi
>>>>>>
>>>>>>> c();
>>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>>                  public int partition() {
>>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>>            };
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()`
>>> in
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> IMO.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
>>>>>>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> amount
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> methods and it is logical to enable inheritance between
>>>>>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> two.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> below 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 has gone
>>>>>>>>>>>>>>>>>>>>>>>> though
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>
>>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> internal
>>>>>>
>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
>>> is
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Streams
>>>>>>
>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>>>>> ka/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
>>>>>
>>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>
> 


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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I just want to point out that the basic idea is great and that we should
apply optimizations like "filter first" and other. But we should NOT
convolute this KIP with orthogonal improvements.

In fact, we have an umbrella JIRA for DSL optimization already:
https://issues.apache.org/jira/browse/KAFKA-6034

@Jan: feel free to create sub-tasks for new optimization ideas and we
can take it from there.


-Matthias


On 12/18/17 7:55 AM, Bill Bejeck wrote:
> Jan,
> 
> I apologize for the delayed response.
> 
> my suggestion would be that instead of
>>
>> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>>
>> we build
>>
>> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
> 
> 
> I agree that filtering before the KTable source makes sense and would be a
> positive change to implement.
> 
> But the situation above is just one scenario out of many we need to
> consider.  I'm not sure we can cover all the implications from different
> use cases ahead of time.
> 
> So I'm inclined to agree with Guozhang that we come up with clear "rules"
> (I use the word rules for lack of a better term) for RecordContext usage
> and inheritance. That way going forward we can have distinct expectations
> of different use cases.
> 
> -Bill
> 
> On Fri, Dec 15, 2017 at 3:57 PM, Guozhang Wang <wa...@gmail.com> wrote:
> 
>> Regarding the record context inheritance: I agree it may be a better idea
>> for now to drop the information when we cannot come up with a consensus
>> about how the record context should be inherited. Like Bill I was a bit
>> worried about the lacking of such data lineage information for trouble
>> shooting in operations or debugging in coding; but I think we can still try
>> to come up with better solutions in the future by extending the current
>> protocol, than coming up with something that we realized that we need to
>> change in the future.
>>
>> Regarding the table / filter question: I agree with Jan that we could
>> consider update the builder so that we will push down the filter earlier
>> than KTable source that materialized the store; on the other hand, I think
>> Matthias' point is that even doing this does not completely exclude the
>> scenarios that you'd have the old/new pairs in your Tables, for example,
>> consider:
>>
>> table1 = stream1.groupBy(...).aggregate(...)
>> table2 = table1.filter(..., Materialized.as(...))
>>
>> In this case table2 is filtering on table1 which is not read from the
>> source, and hence it already outputs the old/new pairs already, so we still
>> need to consider how to handle it.
>>
>>
>> So I'd suggest the following execution plan towards KIP-159:
>>
>> 1) revisit our record context (topic, partition, offset, timestamp)
>> protocols that is used at the DSL layer, make it clear at which high-level
>> operators we should apply certain inheritance rule, and which others we
>> should drop such information.
>>     1.1) modify the lower-level PAPI that DSL leverages, to allow the
>> caller (DSL) to modify the record context (note that today for lower-level
>> API, the record context is always passed through when forwarding to the
>> next processor node)
>> 2) at the same time, consider optimizing the source KTable filter cases (I
>> think we already have certain JIRA tickets for this) so that the filter
>> operator is pushed early than the KTABLESOURCE node where materialization
>> happens.
>> 3) after 1) is done, come back to KIP-159 and add the proposed APIs.
>>
>>
>> Guozhang
>>
>>
>> On Thu, Dec 7, 2017 at 12:27 PM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>>> Thank you Bill,
>>>
>>> I think this is reasonable. Do you have any suggestion
>>> for handling oldValues in cases like
>>>
>>> builder.table().filter(RichPredicate).join()
>>>
>>> where we process a Change with old and new value and dont have a record
>>> context for old.
>>>
>>> my suggestion would be that instead of
>>>
>>> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>>>
>>> we build
>>>
>>> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
>>>
>>> We should build a topology like this from the beginning and not have
>>> an optimisation phase afterwards.
>>>
>>> Any opinions?
>>>
>>> Best Jan
>>>
>>>
>>>
>>>
>>> On 05.12.2017 17:34, Bill Bejeck wrote:
>>>
>>>> Matthias,
>>>>
>>>> Overall I agree with what you've presented here.
>>>>
>>>> Initially, I was hesitant to remove information from the context of the
>>>> result records (Joins or Aggregations) with the thought that when there
>>>> are
>>>> unexpected results, the source information would be useful for tracing
>>>> back
>>>> where the error could have occurred.  But in the case of Joins and
>>>> Aggregations, the amount of data needed to do meaningful analysis could
>> be
>>>> too much. For example, a join result could come from two topics so you'd
>>>> need to keep both original topic names, offsets, etc. (plus the broker
>>>> could have deleted the records in the interim so even having offset
>> could
>>>> provide nothing).
>>>>
>>>> I'm bit long winded here, but I've come full circle to your original
>>>> proposal that since Joins and Aggregations produce fundamentally new
>>>> types,
>>>> we drop the corresponding information from the context even in the case
>> of
>>>> single topic aggregations.
>>>>
>>>> Thanks,
>>>> Bill
>>>>
>>>> On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>> I agree with Guozhang that just exposing meta data at the source level
>>>>> might not provide too much value. Furthermore, for timestamps we do
>>>>> already have a well defined contract and we should exploit it:
>>>>> timestamps can always be provided in a meaningful way.
>>>>>
>>>>> Also, for simple operations like KStream-filter/map the contract is
>>>>> simple and we can just use it. Same for KTable-filter/map (for new
>>>>> values).
>>>>>
>>>>> For aggregations, join, and oldValue, I could just drop some
>> information
>>>>> and return `null`/-1, if the result records has no semantically
>>>>> meaningful meta data.
>>>>>
>>>>> For example, for aggregations, we could preserve the partition (as all
>>>>> agg-input-records have the same partition). For single input topic
>>>>> aggregation (what I guess is the most prominent case), we can also
>> carry
>>>>> over the topic name (would be a internal repartitioning topic name
>>>>> often). Offsets don't have any semantic interpretation IMHO and we
>> could
>>>>> return -1.
>>>>>
>>>>> For joins, we could keep the partition information. Topic and offset
>> are
>>>>> both unknown/invalid for the output record IMHO.
>>>>>
>>>>> For the oldValue case, we can keep partition and for single input topic
>>>>> case topic name. Timestamp might be -1 for now, but after we added
>>>>> timestamps to KTable (what we plan to do anyway), we can also return a
>>>>> valid timestamp. Offset would be -1 again (if we store offset in KTable
>>>>> too, we could provide all offset as well -- but I don't see too much
>>>>> value in doing this compared to the storage overhead this implies).
>>>>>
>>>>>
>>>>> WDYT?
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 11/29/17 4:14 AM, Jan Filipiak wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> thank you for the summary and thanks for acknowledging that I do have
>> a
>>>>>> point here.
>>>>>>
>>>>>> I don't like the second Idea at all. Hence I started of this
>> discussion.
>>>>>>
>>>>>> I am just disappointed, back then when we had the discussion about how
>>>>>> to refactor store overload
>>>>>> and IQ handling, I knew the path we are taking is wrong. Having
>> problems
>>>>>> implementing these kinda
>>>>>> features (wich are really simple)  is just a symptom of messed up IQ
>>>>>> implementation. I wish really bad
>>>>>> I could have convinced you guys back then. To be honest with IQ we can
>>>>>> continue here
>>>>>> as we Materialize but would not send oldValue, but with join you're
>> out
>>>>>> of luck with current setup.
>>>>>>
>>>>>> I of course recommend to do not introduce any optimizations here. Id
>>>>>> recommend to go towards what
>>>>>> I recommended already back then. So i would't say we need to optimize
>>>>>> anything later we need to build
>>>>>> the topology better in the first place.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 28.11.2017 21:00, Guozhang Wang wrote:
>>>>>>
>>>>>>> Jan,
>>>>>>>
>>>>>>> Thanks for your input, I can understand now that the oldValue is also
>>>>>>> exposed in user customized `filter` function and hence want record
>>>>>>> context
>>>>>>> we should expose is a problem. And I think it does brings a good
>> point
>>>>>>>
>>>>>> to
>>>>>
>>>>>> consider for KIP-159. The discussions maybe a bit confusing to reader
>>>>>>> though, and hence I'd like to summarize the status quo and with a
>>>>>>> proposal:
>>>>>>>
>>>>>>> In today's Streams DSL, when a KTable is created either from a source
>>>>>>> topic, or from an stateful operator, we will materialize the KTable
>>>>>>> with a
>>>>>>> backing state store; on the other hand, KTables created from a
>>>>>>> non-stateful
>>>>>>> operator like filter, will not be backed by a state store by default
>>>>>>> unless
>>>>>>> users indicate so (e.g. using the overloaded function with the
>>>>>>> queryable
>>>>>>> name or store supplier).
>>>>>>>
>>>>>>> For example:
>>>>>>>
>>>>>>> KTable table1 = builder.table("topic");
>>>>>>>
>>>>>> // a
>>>>>
>>>>>> state store created for table1
>>>>>>> KTable table2 = table1.filter(..);
>>>>>>> // no state store created for table2
>>>>>>> KTable table3 = table1.filter(.., "storeName");                  // a
>>>>>>> state
>>>>>>> store created for table3
>>>>>>> KTable table4 = table1.groupBy(..).aggregate(..);            // a
>>>>>>> state
>>>>>>> store created for table4
>>>>>>>
>>>>>>> Because of that, the filter() operator above on table1 will always be
>>>>>>> exposed with oldValue and newValue; Damian's point is that, we may
>>>>>>> optimize
>>>>>>> the first case such that table1 will only be materialized if users
>>>>>>> asked so
>>>>>>> (e.g. using the overloaded function with a store supplier), and in
>>>>>>> which
>>>>>>> case, we do not need to pass newValue / oldValue pairs (I think this
>> is
>>>>>>> what Jan suggests as well, i.e. do filtering before materializing, so
>>>>>>> that
>>>>>>> we can have a smaller backed state store as well). But this
>>>>>>> optimization
>>>>>>> does not eliminate the possibilities that we may still need to do
>>>>>>> filter if
>>>>>>> users does specify "yes I do want to the source KTable itself to be
>>>>>>> materialized, please". So the concern about how to expose the record
>>>>>>> context in such cases still persists.
>>>>>>>
>>>>>>>
>>>>>>> With that, regarding to KIP-159 itself, here are my thoughts:
>>>>>>>
>>>>>>> 1) if we restrict the scope of exposing record context only to source
>>>>>>> KTables / KStreams I felt the KIP itself does not bring much value
>>>>>>> given
>>>>>>> its required API change because only the SourceKStream can safely
>>>>>>> maintain
>>>>>>> its records context, and for SourceKTable if it is materialized, then
>>>>>>> even
>>>>>>> non-stateful operators like Join may still have a concern about
>>>>>>> exposing
>>>>>>> the record context.
>>>>>>>
>>>>>>> 2) an alternative idea is we provide the semantics on how record
>>>>>>> context
>>>>>>> would be inherited across the operators for KTable / KStream and
>>>>>>> expose it
>>>>>>> in all operators (similarly in PAPI we would expose a much simpler
>>>>>>> contract), and make it as a public contract that Streams library will
>>>>>>> guarantee moving forward even we optimize our topology builder; it
>> may
>>>>>>> not
>>>>>>> align perfectly with the linear algebraic semantics but practically
>>>>>>> applicable for most cases; if users semantics do not fit in the
>>>>>>> provided
>>>>>>> contract, then they may need to get this themselves (embed such
>>>>>>> information
>>>>>>> in the value payload, for example).
>>>>>>>
>>>>>>> If people do not like the second idea, I'd suggest we hold on
>> pursuing
>>>>>>> the
>>>>>>> first direction since to me its beneficial scope is too limited
>>>>>>> compared to
>>>>>>> its cost.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>> wrote:
>>>>>>>
>>>>>>> Cleary we show the oldValue to the user. We have to, because we
>> filter
>>>>>>>> after the store.
>>>>>>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>>>>>>>> ain/java/org/apache/kafka/streams/kstream/internals/
>>>>>>>>
>>>>>>> KTableFilter.java#L96
>>>>>
>>>>>>
>>>>>>>> I cannot help you following this. It is really obvious and I am
>>>>>>>> running
>>>>>>>> out of tools for explaining.
>>>>>>>>
>>>>>>>> Thanks for understanding my point to put filter before. Not only
>>>>>>>> would it
>>>>>>>> make the store smaller. It would make this feature reasonably
>>>>>>>> possible and
>>>>>>>> the framework easier. Interestingly it would also help to move IQ
>>>>>>>> into more
>>>>>>>> reasonable directions. And it might help understand that we do not
>>>>>>>> need any
>>>>>>>> intermediate representation of the topology,
>>>>>>>>
>>>>>>>> KIP-182 I have no clue what everyone has with their "bytestores" so
>>>>>>>> broken. But putting another store after doesn't help when the store
>>>>>>>> before
>>>>>>>> is the problem.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>>>>>>>
>>>>>>>>    From a DSL point of view, users only see the new value on a
>>>>>>>>> KTable#filter anyway. So why should it be an issue that we use
>>>>>>>>> <newValue,oldValue> pair under the hood?
>>>>>>>>>
>>>>>>>>> User sees newValue and gets corresponding RecordContext. I can't
>> see
>>>>>>>>> any
>>>>>>>>> issue here?
>>>>>>>>>
>>>>>>>>> I cannot follow here:
>>>>>>>>>
>>>>>>>>> Even when we have a statefull operation last. We move it to the
>> very
>>>>>>>>>
>>>>>>>>>> first processor (KtableSource)
>>>>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>>>>
>>>>>>>>>>> With regard to `builder.table().filter()`:
>>>>>>>>>
>>>>>>>>> I see you point that it would be good to be able to apply the
>>>>>>>>> filter()
>>>>>>>>> first to reduce the stat store size of the table. But how is this
>>>>>>>>> related to KIP-159?
>>>>>>>>>
>>>>>>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>>>>>>>> putting a custom dummy store into the table and materialize the
>>>>>>>>> filter
>>>>>>>>> result afterwards? It's not a nice way to do, but seems to be
>>>>>>>>>
>>>>>>>> possible.
>>>>>
>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>>>>>>>
>>>>>>>>> The comment is valid. It falls exactly into this topic, it has
>>>>>>>>>>
>>>>>>>>> exactly
>>>>>
>>>>>> todo with this!
>>>>>>>>>> Even when we have a statefull operation last. We move it to the
>> very
>>>>>>>>>> first processor (KtableSource)
>>>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>>>
>>>>>>>>>> Regarding the other Jiras you are referring to. They harm the
>>>>>>>>>> project
>>>>>>>>>> more than they do good!
>>>>>>>>>> There is no need for this kind of optimizer and meta
>> representation
>>>>>>>>>> and
>>>>>>>>>> what not. I hope they
>>>>>>>>>> never get implemented.
>>>>>>>>>>
>>>>>>>>>> Best Jan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>>>>>>>
>>>>>>>>>> Jan, i think you comment with respect to filtering is valid,
>> though
>>>>>>>>>>> not for
>>>>>>>>>>> this KIP. We have separate JIRAs for topology optimization of
>>>>>>>>>>> which this
>>>>>>>>>>> falls into.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Jan,
>>>>>>>>>>>
>>>>>>>>>>>> Not sure I understand your argument that "we still going to
>>>>>>>>>>>> present
>>>>>>>>>>>> change.oldValue to the filter even though the record context()
>> is
>>>>>>>>>>>> for
>>>>>>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
>>>>>>>>>>>> If yes
>>>>>>>>>>>> could you point to me which LOC are you concerning about?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> a remark of mine that got missed during migration:
>>>>>>>>>>>>
>>>>>>>>>>>>> There is this problem that even though we have
>>>>>>>>>>>>> source.table.filter.join
>>>>>>>>>>>>> the state-fullness happens at the table step not a the join
>>>>>>>>>>>>> step. In a
>>>>>>>>>>>>> filter
>>>>>>>>>>>>> we still going to present change.oldValue to the filter even
>>>>>>>>>>>>>
>>>>>>>>>>>> though
>>>>>
>>>>>> the
>>>>>>>>>>>>> record context() is for change.newValue. I would go as far as
>>>>>>>>>>>>> applying
>>>>>>>>>>>>> the filter before the table processor. Not to just get KIP-159,
>>>>>>>>>>>>>
>>>>>>>>>>>> but
>>>>>
>>>>>> because
>>>>>>>>>>>>
>>>>>>>>>>>> I think its a side effect of a non ideal topology layout. If i
>> can
>>>>>>>>>>>>> filter
>>>>>>>>>>>>> 99% of my
>>>>>>>>>>>>> records. my state could be way smaller. Also widely escalates
>> the
>>>>>>>>>>>>> context
>>>>>>>>>>>>> of the KIP
>>>>>>>>>>>>>
>>>>>>>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails
>>>>>>>>>>>>> were
>>>>>>>>>>>>>
>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>> to VOTE thread.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Copying Guozhang's last summary below. Thanks for this
>> summary.
>>>>>>>>>>>>>> Very
>>>>>>>>>>>>>> comprehensive!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>>>>>>>> context
>>>>>>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to
>> (2)
>>>>>>>>>>>>>> carrying the context to more operators than just sources. It
>>>>>>>>>>>>>> also
>>>>>>>>>>>>>> seems,
>>>>>>>>>>>>>> that we all agree, that many-to-one operations void the
>> context.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I still think, that just going with plain (2) is too
>>>>>>>>>>>>>> restrictive --
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Also note, that the two operators filter() and filterNot()
>> don't
>>>>>>>>>>>>>> modify
>>>>>>>>>>>>>> the record and thus for both, it would be absolutely valid to
>>>>>>>>>>>>>>
>>>>>>>>>>>>> keep
>>>>>
>>>>>> the
>>>>>>>>>>>>>> context.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I personally would keep the context for at least all
>> one-to-one
>>>>>>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
>>>>>>>>>>>>>>
>>>>>>>>>>>>> the
>>>>>
>>>>>> context further: at least the offset information is
>>>>>>>>>>>>>> questionable for
>>>>>>>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>>>>>>>> inherited
>>>>>>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>>>>>>>> "partition". Thus, I think it's still valuable information we
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>> carry
>>>>>>>>>>>>>> downstreams.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Jan: which approach are you referring to as "the approach that
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> table would be perfect"?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>>>>>>>> exposing the
>>>>>>>>>>>>>>> record context which has the issues that we have been
>>>>>>>>>>>>>>> discussing
>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>> and its semantics is always referring to the "processing
>>>>>>>>>>>>>>> record" at
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>
>>>>>>>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>>>>>>
>>>>>>>>>>>>>> different:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1) the record traversed the topology from source to sink, it
>>>>>>>>>>>>>>> may be
>>>>>>>>>>>>>>> transformed into new object or even generate multiple new
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> objects
>>>>>
>>>>>> (think:
>>>>>>>>>>>>>>
>>>>>>>>>>>>> branch) along the traversal. And the record context is
>> referring
>>>>>>>>>>>>>
>>>>>>>>>>>> to
>>>>>
>>>>>> this
>>>>>>>>>>>>>>
>>>>>>>>>>>>> processing record. Here the "lifetime" of the record lasts for
>>>>>>>>>>>>> the
>>>>>>>>>>>>>
>>>>>>>>>>>>>> entire
>>>>>>>>>>>>>>
>>>>>>>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>>>>>>
>>>>>>>>>>>>>> treated as
>>>>>>>>>>>>>>> different transformed values of this record (this applies to
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> join
>>>>>
>>>>>> and
>>>>>>>>>>>>>>> aggregations as well).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2) the record being processed is wiped out in the first
>>>>>>>>>>>>>>> operator
>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> source, and NEW records are forwarded to downstream
>> operators.
>>>>>>>>>>>>>>> I.e.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>
>>>>>>>>>>>>> record only lives between two adjacent operators, once it
>>>>>>>>>>>>> reached the
>>>>>>>>>>>>>
>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>
>>>>>>>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think in the past we have talked about Streams under both
>>>>>>>>>>>>>>> context,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>
>>>>>>>>>>>>> we
>>>>>>>>>>>>>
>>>>>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically
>> more
>>>>>>>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>
>>>>>>>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>>>>>>
>>>>>>>>>>>>>> ends at
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> join operator as it is only be materialized, while stream
>>>>>>>>>>>>>>> record's
>>>>>>>>>>>>>>> traversal goes through the join operator to further down
>> until
>>>>>>>>>>>>>>> sinks).
>>>>>>>>>>>>>>> However if we are going to interpret following 2) above then
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> even
>>>>>
>>>>>> for
>>>>>>>>>>>>>>> non-stateful operators we would not inherit record context.
>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>> we're
>>>>>>>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3) a record would traverse "through" one-to-one
>> (non-stateful)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> operators,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> will "replicate" at one-to-many (non-stateful) operators
>> (think:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> "mapValues"
>>>>>>>>>>>>>>>       ) and will "end" at many-to-one (stateful) operators
>>>>>>>>>>>>>>> where NEW
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>
>>>>>>>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>> page before chatting more.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
>>>>>>>>>>>>>>> designs
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>
>>>>>>>>>>>>> punctuate() was not deprecated.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I corrected.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I just re-read the KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed
>> completely
>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Jeyhun,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> carefully if we should add this functionality to the DSL
>> layer
>>>>>>>>>>>>>
>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>>>>>>>> conclusion is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> it would require revamping the public APIs quite a lot, and
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> it's
>>>>>
>>>>>> not
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> clear
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> if it is a good trade-off than asking users to call
>> process()
>>>>>>>>>>>>>>>> instead.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
>>>>>>>>>>>>>>>>> <da...@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         - on-demand commit() feature
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
>>>>>>>>>>>>>>>>>> simplified
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
>>>>>>>>>>>>>>>>>>> agree that
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
>>>>>>>>>>>>>>>>>>>> close the
>>>>>>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> best
>>>>>
>>>>>> way
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> do.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> A
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
>>>>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>
>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> that
>>>>>
>>>>>> flexibility
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains
>> processor
>>>>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context
>>>>>>>>>>>>>>>>>>>>> (offset,
>>>>>>>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want
>> to
>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call
>> from
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion
>> of
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>>>>>>> `commit()`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>
>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
>>>>>>>>>>>>>>>>>>>> elaborate a
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that
>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
>>>>>>>>>>>>>>>>>>>> better. Why
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit?
>> To
>>>>>>>>>>>>>
>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> explicitly,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my
>> understanding,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> is an helper object that provide access to record meta
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> data.
>>>>>
>>>>>> Requesting
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> specific
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
>>>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> path.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>>
>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> internal
>>>>>
>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
>> is
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
>>>>>>>>>>>>>>>>>>>>> `RecordContext` is
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> interfaces.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>> wants to commit, then there should be some method
>> inside
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
>>>>>>>>>>>>>>>>>>>>>>> snippet in
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>          public void process(final K1 key, final V1
>>>>>>>>>>>>>>>>>>>> value) {
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>              recordContext = new RecordContext()
>>>>>>>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>                  public void commit() {
>>>>>>>>>>>>>>>>>>>>>>>                      context().commit();
>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>                  public long offset() {
>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> context().recordContext().offs
>>>>>
>>>>>> et();
>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>                  public long timestamp() {
>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>                  public String topic() {
>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> context().recordContext().topi
>>>>>
>>>>>> c();
>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>>>                  public int partition() {
>>>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>>>            };
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()`
>> in
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> IMO.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
>>>>>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> amount
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>
>>>>>>>>>>>>>> methods and it is logical to enable inheritance between
>>>>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> two.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> below 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 has gone
>>>>>>>>>>>>>>>>>>>>>>> though
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>
>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> about
>>>>>
>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>>
>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> internal
>>>>>
>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
>> is
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Streams
>>>>>
>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>>>> ka/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
>>>>
>>>>
>>
>>
>> --
>> -- Guozhang
>>
> 


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

Posted by Bill Bejeck <bb...@gmail.com>.
Jan,

I apologize for the delayed response.

my suggestion would be that instead of
>
> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>
> we build
>
> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK


I agree that filtering before the KTable source makes sense and would be a
positive change to implement.

But the situation above is just one scenario out of many we need to
consider.  I'm not sure we can cover all the implications from different
use cases ahead of time.

So I'm inclined to agree with Guozhang that we come up with clear "rules"
(I use the word rules for lack of a better term) for RecordContext usage
and inheritance. That way going forward we can have distinct expectations
of different use cases.

-Bill

On Fri, Dec 15, 2017 at 3:57 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Regarding the record context inheritance: I agree it may be a better idea
> for now to drop the information when we cannot come up with a consensus
> about how the record context should be inherited. Like Bill I was a bit
> worried about the lacking of such data lineage information for trouble
> shooting in operations or debugging in coding; but I think we can still try
> to come up with better solutions in the future by extending the current
> protocol, than coming up with something that we realized that we need to
> change in the future.
>
> Regarding the table / filter question: I agree with Jan that we could
> consider update the builder so that we will push down the filter earlier
> than KTable source that materialized the store; on the other hand, I think
> Matthias' point is that even doing this does not completely exclude the
> scenarios that you'd have the old/new pairs in your Tables, for example,
> consider:
>
> table1 = stream1.groupBy(...).aggregate(...)
> table2 = table1.filter(..., Materialized.as(...))
>
> In this case table2 is filtering on table1 which is not read from the
> source, and hence it already outputs the old/new pairs already, so we still
> need to consider how to handle it.
>
>
> So I'd suggest the following execution plan towards KIP-159:
>
> 1) revisit our record context (topic, partition, offset, timestamp)
> protocols that is used at the DSL layer, make it clear at which high-level
> operators we should apply certain inheritance rule, and which others we
> should drop such information.
>     1.1) modify the lower-level PAPI that DSL leverages, to allow the
> caller (DSL) to modify the record context (note that today for lower-level
> API, the record context is always passed through when forwarding to the
> next processor node)
> 2) at the same time, consider optimizing the source KTable filter cases (I
> think we already have certain JIRA tickets for this) so that the filter
> operator is pushed early than the KTABLESOURCE node where materialization
> happens.
> 3) after 1) is done, come back to KIP-159 and add the proposed APIs.
>
>
> Guozhang
>
>
> On Thu, Dec 7, 2017 at 12:27 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > Thank you Bill,
> >
> > I think this is reasonable. Do you have any suggestion
> > for handling oldValues in cases like
> >
> > builder.table().filter(RichPredicate).join()
> >
> > where we process a Change with old and new value and dont have a record
> > context for old.
> >
> > my suggestion would be that instead of
> >
> > SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
> >
> > we build
> >
> > SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
> >
> > We should build a topology like this from the beginning and not have
> > an optimisation phase afterwards.
> >
> > Any opinions?
> >
> > Best Jan
> >
> >
> >
> >
> > On 05.12.2017 17:34, Bill Bejeck wrote:
> >
> >> Matthias,
> >>
> >> Overall I agree with what you've presented here.
> >>
> >> Initially, I was hesitant to remove information from the context of the
> >> result records (Joins or Aggregations) with the thought that when there
> >> are
> >> unexpected results, the source information would be useful for tracing
> >> back
> >> where the error could have occurred.  But in the case of Joins and
> >> Aggregations, the amount of data needed to do meaningful analysis could
> be
> >> too much. For example, a join result could come from two topics so you'd
> >> need to keep both original topic names, offsets, etc. (plus the broker
> >> could have deleted the records in the interim so even having offset
> could
> >> provide nothing).
> >>
> >> I'm bit long winded here, but I've come full circle to your original
> >> proposal that since Joins and Aggregations produce fundamentally new
> >> types,
> >> we drop the corresponding information from the context even in the case
> of
> >> single topic aggregations.
> >>
> >> Thanks,
> >> Bill
> >>
> >> On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
> >> wrote:
> >>
> >> I agree with Guozhang that just exposing meta data at the source level
> >>> might not provide too much value. Furthermore, for timestamps we do
> >>> already have a well defined contract and we should exploit it:
> >>> timestamps can always be provided in a meaningful way.
> >>>
> >>> Also, for simple operations like KStream-filter/map the contract is
> >>> simple and we can just use it. Same for KTable-filter/map (for new
> >>> values).
> >>>
> >>> For aggregations, join, and oldValue, I could just drop some
> information
> >>> and return `null`/-1, if the result records has no semantically
> >>> meaningful meta data.
> >>>
> >>> For example, for aggregations, we could preserve the partition (as all
> >>> agg-input-records have the same partition). For single input topic
> >>> aggregation (what I guess is the most prominent case), we can also
> carry
> >>> over the topic name (would be a internal repartitioning topic name
> >>> often). Offsets don't have any semantic interpretation IMHO and we
> could
> >>> return -1.
> >>>
> >>> For joins, we could keep the partition information. Topic and offset
> are
> >>> both unknown/invalid for the output record IMHO.
> >>>
> >>> For the oldValue case, we can keep partition and for single input topic
> >>> case topic name. Timestamp might be -1 for now, but after we added
> >>> timestamps to KTable (what we plan to do anyway), we can also return a
> >>> valid timestamp. Offset would be -1 again (if we store offset in KTable
> >>> too, we could provide all offset as well -- but I don't see too much
> >>> value in doing this compared to the storage overhead this implies).
> >>>
> >>>
> >>> WDYT?
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 11/29/17 4:14 AM, Jan Filipiak wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> thank you for the summary and thanks for acknowledging that I do have
> a
> >>>> point here.
> >>>>
> >>>> I don't like the second Idea at all. Hence I started of this
> discussion.
> >>>>
> >>>> I am just disappointed, back then when we had the discussion about how
> >>>> to refactor store overload
> >>>> and IQ handling, I knew the path we are taking is wrong. Having
> problems
> >>>> implementing these kinda
> >>>> features (wich are really simple)  is just a symptom of messed up IQ
> >>>> implementation. I wish really bad
> >>>> I could have convinced you guys back then. To be honest with IQ we can
> >>>> continue here
> >>>> as we Materialize but would not send oldValue, but with join you're
> out
> >>>> of luck with current setup.
> >>>>
> >>>> I of course recommend to do not introduce any optimizations here. Id
> >>>> recommend to go towards what
> >>>> I recommended already back then. So i would't say we need to optimize
> >>>> anything later we need to build
> >>>> the topology better in the first place.
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 28.11.2017 21:00, Guozhang Wang wrote:
> >>>>
> >>>>> Jan,
> >>>>>
> >>>>> Thanks for your input, I can understand now that the oldValue is also
> >>>>> exposed in user customized `filter` function and hence want record
> >>>>> context
> >>>>> we should expose is a problem. And I think it does brings a good
> point
> >>>>>
> >>>> to
> >>>
> >>>> consider for KIP-159. The discussions maybe a bit confusing to reader
> >>>>> though, and hence I'd like to summarize the status quo and with a
> >>>>> proposal:
> >>>>>
> >>>>> In today's Streams DSL, when a KTable is created either from a source
> >>>>> topic, or from an stateful operator, we will materialize the KTable
> >>>>> with a
> >>>>> backing state store; on the other hand, KTables created from a
> >>>>> non-stateful
> >>>>> operator like filter, will not be backed by a state store by default
> >>>>> unless
> >>>>> users indicate so (e.g. using the overloaded function with the
> >>>>> queryable
> >>>>> name or store supplier).
> >>>>>
> >>>>> For example:
> >>>>>
> >>>>> KTable table1 = builder.table("topic");
> >>>>>
> >>>> // a
> >>>
> >>>> state store created for table1
> >>>>> KTable table2 = table1.filter(..);
> >>>>> // no state store created for table2
> >>>>> KTable table3 = table1.filter(.., "storeName");                  // a
> >>>>> state
> >>>>> store created for table3
> >>>>> KTable table4 = table1.groupBy(..).aggregate(..);            // a
> >>>>> state
> >>>>> store created for table4
> >>>>>
> >>>>> Because of that, the filter() operator above on table1 will always be
> >>>>> exposed with oldValue and newValue; Damian's point is that, we may
> >>>>> optimize
> >>>>> the first case such that table1 will only be materialized if users
> >>>>> asked so
> >>>>> (e.g. using the overloaded function with a store supplier), and in
> >>>>> which
> >>>>> case, we do not need to pass newValue / oldValue pairs (I think this
> is
> >>>>> what Jan suggests as well, i.e. do filtering before materializing, so
> >>>>> that
> >>>>> we can have a smaller backed state store as well). But this
> >>>>> optimization
> >>>>> does not eliminate the possibilities that we may still need to do
> >>>>> filter if
> >>>>> users does specify "yes I do want to the source KTable itself to be
> >>>>> materialized, please". So the concern about how to expose the record
> >>>>> context in such cases still persists.
> >>>>>
> >>>>>
> >>>>> With that, regarding to KIP-159 itself, here are my thoughts:
> >>>>>
> >>>>> 1) if we restrict the scope of exposing record context only to source
> >>>>> KTables / KStreams I felt the KIP itself does not bring much value
> >>>>> given
> >>>>> its required API change because only the SourceKStream can safely
> >>>>> maintain
> >>>>> its records context, and for SourceKTable if it is materialized, then
> >>>>> even
> >>>>> non-stateful operators like Join may still have a concern about
> >>>>> exposing
> >>>>> the record context.
> >>>>>
> >>>>> 2) an alternative idea is we provide the semantics on how record
> >>>>> context
> >>>>> would be inherited across the operators for KTable / KStream and
> >>>>> expose it
> >>>>> in all operators (similarly in PAPI we would expose a much simpler
> >>>>> contract), and make it as a public contract that Streams library will
> >>>>> guarantee moving forward even we optimize our topology builder; it
> may
> >>>>> not
> >>>>> align perfectly with the linear algebraic semantics but practically
> >>>>> applicable for most cases; if users semantics do not fit in the
> >>>>> provided
> >>>>> contract, then they may need to get this themselves (embed such
> >>>>> information
> >>>>> in the value payload, for example).
> >>>>>
> >>>>> If people do not like the second idea, I'd suggest we hold on
> pursuing
> >>>>> the
> >>>>> first direction since to me its beneficial scope is too limited
> >>>>> compared to
> >>>>> its cost.
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <
> >>>>> Jan.Filipiak@trivago.com
> >>>>> wrote:
> >>>>>
> >>>>> Cleary we show the oldValue to the user. We have to, because we
> filter
> >>>>>> after the store.
> >>>>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
> >>>>>> ain/java/org/apache/kafka/streams/kstream/internals/
> >>>>>>
> >>>>> KTableFilter.java#L96
> >>>
> >>>>
> >>>>>> I cannot help you following this. It is really obvious and I am
> >>>>>> running
> >>>>>> out of tools for explaining.
> >>>>>>
> >>>>>> Thanks for understanding my point to put filter before. Not only
> >>>>>> would it
> >>>>>> make the store smaller. It would make this feature reasonably
> >>>>>> possible and
> >>>>>> the framework easier. Interestingly it would also help to move IQ
> >>>>>> into more
> >>>>>> reasonable directions. And it might help understand that we do not
> >>>>>> need any
> >>>>>> intermediate representation of the topology,
> >>>>>>
> >>>>>> KIP-182 I have no clue what everyone has with their "bytestores" so
> >>>>>> broken. But putting another store after doesn't help when the store
> >>>>>> before
> >>>>>> is the problem.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
> >>>>>>
> >>>>>>    From a DSL point of view, users only see the new value on a
> >>>>>>> KTable#filter anyway. So why should it be an issue that we use
> >>>>>>> <newValue,oldValue> pair under the hood?
> >>>>>>>
> >>>>>>> User sees newValue and gets corresponding RecordContext. I can't
> see
> >>>>>>> any
> >>>>>>> issue here?
> >>>>>>>
> >>>>>>> I cannot follow here:
> >>>>>>>
> >>>>>>> Even when we have a statefull operation last. We move it to the
> very
> >>>>>>>
> >>>>>>>> first processor (KtableSource)
> >>>>>>>>> and therefore cant present a proper RecordContext.
> >>>>>>>>>
> >>>>>>>>> With regard to `builder.table().filter()`:
> >>>>>>>
> >>>>>>> I see you point that it would be good to be able to apply the
> >>>>>>> filter()
> >>>>>>> first to reduce the stat store size of the table. But how is this
> >>>>>>> related to KIP-159?
> >>>>>>>
> >>>>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
> >>>>>>> putting a custom dummy store into the table and materialize the
> >>>>>>> filter
> >>>>>>> result afterwards? It's not a nice way to do, but seems to be
> >>>>>>>
> >>>>>> possible.
> >>>
> >>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
> >>>>>>>
> >>>>>>> The comment is valid. It falls exactly into this topic, it has
> >>>>>>>>
> >>>>>>> exactly
> >>>
> >>>> todo with this!
> >>>>>>>> Even when we have a statefull operation last. We move it to the
> very
> >>>>>>>> first processor (KtableSource)
> >>>>>>>> and therefore cant present a proper RecordContext.
> >>>>>>>>
> >>>>>>>> Regarding the other Jiras you are referring to. They harm the
> >>>>>>>> project
> >>>>>>>> more than they do good!
> >>>>>>>> There is no need for this kind of optimizer and meta
> representation
> >>>>>>>> and
> >>>>>>>> what not. I hope they
> >>>>>>>> never get implemented.
> >>>>>>>>
> >>>>>>>> Best Jan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 22.11.2017 14:44, Damian Guy wrote:
> >>>>>>>>
> >>>>>>>> Jan, i think you comment with respect to filtering is valid,
> though
> >>>>>>>>> not for
> >>>>>>>>> this KIP. We have separate JIRAs for topology optimization of
> >>>>>>>>> which this
> >>>>>>>>> falls into.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Jan,
> >>>>>>>>>
> >>>>>>>>>> Not sure I understand your argument that "we still going to
> >>>>>>>>>> present
> >>>>>>>>>> change.oldValue to the filter even though the record context()
> is
> >>>>>>>>>> for
> >>>>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
> >>>>>>>>>> If yes
> >>>>>>>>>> could you point to me which LOC are you concerning about?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Guozhang
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
> >>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> a remark of mine that got missed during migration:
> >>>>>>>>>>
> >>>>>>>>>>> There is this problem that even though we have
> >>>>>>>>>>> source.table.filter.join
> >>>>>>>>>>> the state-fullness happens at the table step not a the join
> >>>>>>>>>>> step. In a
> >>>>>>>>>>> filter
> >>>>>>>>>>> we still going to present change.oldValue to the filter even
> >>>>>>>>>>>
> >>>>>>>>>> though
> >>>
> >>>> the
> >>>>>>>>>>> record context() is for change.newValue. I would go as far as
> >>>>>>>>>>> applying
> >>>>>>>>>>> the filter before the table processor. Not to just get KIP-159,
> >>>>>>>>>>>
> >>>>>>>>>> but
> >>>
> >>>> because
> >>>>>>>>>>
> >>>>>>>>>> I think its a side effect of a non ideal topology layout. If i
> can
> >>>>>>>>>>> filter
> >>>>>>>>>>> 99% of my
> >>>>>>>>>>> records. my state could be way smaller. Also widely escalates
> the
> >>>>>>>>>>> context
> >>>>>>>>>>> of the KIP
> >>>>>>>>>>>
> >>>>>>>>>>> I can only see upsides of executing the filter first.
> >>>>>>>>>>>
> >>>>>>>>>>> Best Jan
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails
> >>>>>>>>>>> were
> >>>>>>>>>>>
> >>>>>>>>>>>> sent
> >>>>>>>>>>>> to VOTE thread.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Copying Guozhang's last summary below. Thanks for this
> summary.
> >>>>>>>>>>>> Very
> >>>>>>>>>>>> comprehensive!
> >>>>>>>>>>>>
> >>>>>>>>>>>> It seems, we all agree, that the current implementation of the
> >>>>>>>>>>>> context
> >>>>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to
> (2)
> >>>>>>>>>>>> carrying the context to more operators than just sources. It
> >>>>>>>>>>>> also
> >>>>>>>>>>>> seems,
> >>>>>>>>>>>> that we all agree, that many-to-one operations void the
> context.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I still think, that just going with plain (2) is too
> >>>>>>>>>>>> restrictive --
> >>>>>>>>>>>> but
> >>>>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
> >>>>>>>>>>>>
> >>>>>>>>>>>> Also note, that the two operators filter() and filterNot()
> don't
> >>>>>>>>>>>> modify
> >>>>>>>>>>>> the record and thus for both, it would be absolutely valid to
> >>>>>>>>>>>>
> >>>>>>>>>>> keep
> >>>
> >>>> the
> >>>>>>>>>>>> context.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I personally would keep the context for at least all
> one-to-one
> >>>>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
> >>>>>>>>>>>>
> >>>>>>>>>>> the
> >>>
> >>>> context further: at least the offset information is
> >>>>>>>>>>>> questionable for
> >>>>>>>>>>>> this case -- note thought, that semantically, the timestamp is
> >>>>>>>>>>>> inherited
> >>>>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
> >>>>>>>>>>>> "partition". Thus, I think it's still valuable information we
> >>>>>>>>>>>> can
> >>>>>>>>>>>> carry
> >>>>>>>>>>>> downstreams.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>> Jan: which approach are you referring to as "the approach that
> >>>>>>>>>>>> is
> >>>>>>>>>>>> on the
> >>>>>>>>>>>>
> >>>>>>>>>>>> table would be perfect"?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Note that in today's PAPI layer we are already effectively
> >>>>>>>>>>>>> exposing the
> >>>>>>>>>>>>> record context which has the issues that we have been
> >>>>>>>>>>>>> discussing
> >>>>>>>>>>>>> right
> >>>>>>>>>>>>> now,
> >>>>>>>>>>>>> and its semantics is always referring to the "processing
> >>>>>>>>>>>>> record" at
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> hand.
> >>>>>>>>>>>>
> >>>>>>>>>>> More specifically, we can think of processing a record a bit
> >>>>>>>>>>>
> >>>>>>>>>>>> different:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 1) the record traversed the topology from source to sink, it
> >>>>>>>>>>>>> may be
> >>>>>>>>>>>>> transformed into new object or even generate multiple new
> >>>>>>>>>>>>>
> >>>>>>>>>>>> objects
> >>>
> >>>> (think:
> >>>>>>>>>>>>
> >>>>>>>>>>> branch) along the traversal. And the record context is
> referring
> >>>>>>>>>>>
> >>>>>>>>>> to
> >>>
> >>>> this
> >>>>>>>>>>>>
> >>>>>>>>>>> processing record. Here the "lifetime" of the record lasts for
> >>>>>>>>>>> the
> >>>>>>>>>>>
> >>>>>>>>>>>> entire
> >>>>>>>>>>>>
> >>>>>>>>>>> topology traversal and any new records of this traversal is
> >>>>>>>>>>>
> >>>>>>>>>>>> treated as
> >>>>>>>>>>>>> different transformed values of this record (this applies to
> >>>>>>>>>>>>>
> >>>>>>>>>>>> join
> >>>
> >>>> and
> >>>>>>>>>>>>> aggregations as well).
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 2) the record being processed is wiped out in the first
> >>>>>>>>>>>>> operator
> >>>>>>>>>>>>> after
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>> source, and NEW records are forwarded to downstream
> operators.
> >>>>>>>>>>>>> I.e.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> each
> >>>>>>>>>>>>
> >>>>>>>>>>> record only lives between two adjacent operators, once it
> >>>>>>>>>>> reached the
> >>>>>>>>>>>
> >>>>>>>>>>>> new
> >>>>>>>>>>>>
> >>>>>>>>>>> operator it's lifetime has ended and new records are generated.
> >>>>>>>>>>>
> >>>>>>>>>>>> I think in the past we have talked about Streams under both
> >>>>>>>>>>>>> context,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>
> >>>>>>>>>>> we
> >>>>>>>>>>>
> >>>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically
> more
> >>>>>>>>>>>>> understandable for users as it does not leak any internal
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> implementation
> >>>>>>>>>>>>
> >>>>>>>>>>> details (e.g. for stream-table joins, table record's traversal
> >>>>>>>>>>>
> >>>>>>>>>>>> ends at
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>> join operator as it is only be materialized, while stream
> >>>>>>>>>>>>> record's
> >>>>>>>>>>>>> traversal goes through the join operator to further down
> until
> >>>>>>>>>>>>> sinks).
> >>>>>>>>>>>>> However if we are going to interpret following 2) above then
> >>>>>>>>>>>>>
> >>>>>>>>>>>> even
> >>>
> >>>> for
> >>>>>>>>>>>>> non-stateful operators we would not inherit record context.
> >>>>>>>>>>>>> What
> >>>>>>>>>>>>> we're
> >>>>>>>>>>>>> discussing now, seems to infer a third semantics:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 3) a record would traverse "through" one-to-one
> (non-stateful)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> operators,
> >>>>>>>>>>>>
> >>>>>>>>>>> will "replicate" at one-to-many (non-stateful) operators
> (think:
> >>>>>>>>>>>
> >>>>>>>>>>>> "mapValues"
> >>>>>>>>>>>>>       ) and will "end" at many-to-one (stateful) operators
> >>>>>>>>>>>>> where NEW
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> records
> >>>>>>>>>>>>
> >>>>>>>>>>> will be generated and forwarded to the downstream operators.
> >>>>>>>>>>>
> >>>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>> same
> >>>>>>>>>>>>> page before chatting more.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>> Hi Matthias,
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
> >>>>>>>>>>>>> designs
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> when
> >>>>>>>>>>>>
> >>>>>>>>>>> punctuate() was not deprecated.
> >>>>>>>>>>>
> >>>>>>>>>>>> I corrected.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
> >>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I just re-read the KIP.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
> >>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed
> completely
> >>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>> of introducing it as deprecated.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Otherwise looks good to me.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for being so patient!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Jeyhun,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
> >>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> carefully if we should add this functionality to the DSL
> layer
> >>>>>>>>>>>
> >>>>>>>>>>>> moving
> >>>>>>>>>>>>>>> forward since from what we discovered working on it the
> >>>>>>>>>>>>>>> conclusion is
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> it would require revamping the public APIs quite a lot, and
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> it's
> >>>
> >>>> not
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> clear
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> if it is a good trade-off than asking users to call
> process()
> >>>>>>>>>>>>>> instead.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>
> >>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
> >>>>>>>>>>>>>>> <da...@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Do we need to remove the line that says:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         - on-demand commit() feature
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
> >>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
> >>>>>>>>>>>>>>>> simplified
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> overall design of KIP a lot.
> >>>>>>>>>>>
> >>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
> >>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
> >>>>>>>>>>>>>>>>> agree that
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> but also we need a commit() method
> >>>>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
> >>>>>>>>>>>>>>>>>> close the
> >>>>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Matthias,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> best
> >>>
> >>>> way
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> do.
> >>>>>>>>>>>
> >>>>>>>>>>>> A
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> bit of history behind this design.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
> >>>>>>>>>>>>>>>>>>> itself
> >>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> argument
> >>>>>>>>>>>
> >>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> that
> >>>
> >>>> flexibility
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains
> processor
> >>>>>>>>>>>>>>>>>> level
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
> >>>>>>>>>>>>>>>>>>> need ij
> >>>>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context
> >>>>>>>>>>>>>>>>>>> (offset,
> >>>>>>>>>>>>>>>>>>> timestamp
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want
> to
> >>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> ProcessorContext.commit()
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call
> from
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> ProcessorContext() .
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion
> of
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> KIP/included
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> jiras please let me know.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
> >>>>>>>>>>>>>>>>>> `commit()`
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>
> >>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
> >>>>>>>>>>>>>>>>>> elaborate a
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> exposed
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
> >>>>>>>>>>>>>>>>>> better. Why
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit?
> To
> >>>>>>>>>>>
> >>>>>>>>>>>> me,
> >>>>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> explicitly,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ie,
> >>>>>>>>>>>
> >>>>>>>>>>>> via Processor API.
> >>>>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> that we
> >>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my
> understanding,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `RecordContext`
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> is an helper object that provide access to record meta
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> data.
> >>>
> >>>> Requesting
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
> >>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> specific
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> record.
> >>>>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
> >>>>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> path.
> >>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> misunderstanding.
> >>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
> >>>>>>>>>>>>>>>>>>>>> ProcessorContext,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> enforce
> >>>>>>>>>>>
> >>>>>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> internal
> >>>
> >>>> implementation
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
> is
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> changed
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> this call.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> - I think we should not deprecate
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
> >>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> main
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
> >>>>>>>>>>>>>>>>>>> `RecordContext` is
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> interfaces.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> So
> >>>>>>>>>>>
> >>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>>> wants to commit, then there should be some method
> inside
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `RecordContext`
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
> >>>>>>>>>>>>>>>>>>>>> snippet in
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> KIP-159):
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> @Override
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          public void process(final K1 key, final V1
> >>>>>>>>>>>>>>>>>> value) {
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>              recordContext = new RecordContext()
> >>>>>>>>>>>>>>>>>>>>> {               //
> >>>>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
> >>>>>>>>>>>>>>>>>>>>>                  @Override
> >>>>>>>>>>>>>>>>>>>>>                  public void commit() {
> >>>>>>>>>>>>>>>>>>>>>                      context().commit();
> >>>>>>>>>>>>>>>>>>>>>                  }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>                  @Override
> >>>>>>>>>>>>>>>>>>>>>                  public long offset() {
> >>>>>>>>>>>>>>>>>>>>>                      return
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> context().recordContext().offs
> >>>
> >>>> et();
> >>>>>>>>>>>>>>>>>>>>>                  }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>                  @Override
> >>>>>>>>>>>>>>>>>>>>>                  public long timestamp() {
> >>>>>>>>>>>>>>>>>>>>>                      return
> >>>>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
> >>>>>>>>>>>>>>>>>>>>>                  }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>                  @Override
> >>>>>>>>>>>>>>>>>>>>>                  public String topic() {
> >>>>>>>>>>>>>>>>>>>>>                      return
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> context().recordContext().topi
> >>>
> >>>> c();
> >>>>>>>>>>>>>>>>>>>>>                  }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>                  @Override
> >>>>>>>>>>>>>>>>>>>>>                  public int partition() {
> >>>>>>>>>>>>>>>>>>>>>                      return
> >>>>>>>>>>>>>>>>>>>>> context().recordContext().partition();
> >>>>>>>>>>>>>>>>>>>>>                  }
> >>>>>>>>>>>>>>>>>>>>>            };
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()`
> in
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> IMO.
> >>>>>>>>>>>
> >>>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
> >>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
> >>>>>>>>>>>>>>>>>>>>> `RecordContext()`
> >>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> class of
> >>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
> >>>>>>>>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> amount
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>
> >>>>>>>>>>>> methods and it is logical to enable inheritance between
> >>>>>>>>>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> two.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>> snippet
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> below 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 has gone
> >>>>>>>>>>>>>>>>>>>>> though
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> changes
> >>>>>>>>>>>
> >>>>>>>>>>>> based on user/developer needs, both in
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
> >>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> about
> >>>
> >>>> KAFKA-3907
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> this 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:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> ProcessorContext,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> enforce
> >>>>>>>>>>>
> >>>>>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> internal
> >>>
> >>>> implementation
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl`
> is
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> changed
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> this call.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>> snippet
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> below would need to be updated as well.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> matthias@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.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> -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
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> got
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> approval [2,3].
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
> >>>>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Streams
> >>>
> >>>> [2]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
> >>>>>>>>>>>>>>>>>>>>>>>> ka/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
> >>
> >>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
Regarding the record context inheritance: I agree it may be a better idea
for now to drop the information when we cannot come up with a consensus
about how the record context should be inherited. Like Bill I was a bit
worried about the lacking of such data lineage information for trouble
shooting in operations or debugging in coding; but I think we can still try
to come up with better solutions in the future by extending the current
protocol, than coming up with something that we realized that we need to
change in the future.

Regarding the table / filter question: I agree with Jan that we could
consider update the builder so that we will push down the filter earlier
than KTable source that materialized the store; on the other hand, I think
Matthias' point is that even doing this does not completely exclude the
scenarios that you'd have the old/new pairs in your Tables, for example,
consider:

table1 = stream1.groupBy(...).aggregate(...)
table2 = table1.filter(..., Materialized.as(...))

In this case table2 is filtering on table1 which is not read from the
source, and hence it already outputs the old/new pairs already, so we still
need to consider how to handle it.


So I'd suggest the following execution plan towards KIP-159:

1) revisit our record context (topic, partition, offset, timestamp)
protocols that is used at the DSL layer, make it clear at which high-level
operators we should apply certain inheritance rule, and which others we
should drop such information.
    1.1) modify the lower-level PAPI that DSL leverages, to allow the
caller (DSL) to modify the record context (note that today for lower-level
API, the record context is always passed through when forwarding to the
next processor node)
2) at the same time, consider optimizing the source KTable filter cases (I
think we already have certain JIRA tickets for this) so that the filter
operator is pushed early than the KTABLESOURCE node where materialization
happens.
3) after 1) is done, come back to KIP-159 and add the proposed APIs.


Guozhang


On Thu, Dec 7, 2017 at 12:27 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Thank you Bill,
>
> I think this is reasonable. Do you have any suggestion
> for handling oldValues in cases like
>
> builder.table().filter(RichPredicate).join()
>
> where we process a Change with old and new value and dont have a record
> context for old.
>
> my suggestion would be that instead of
>
> SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK
>
> we build
>
> SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK
>
> We should build a topology like this from the beginning and not have
> an optimisation phase afterwards.
>
> Any opinions?
>
> Best Jan
>
>
>
>
> On 05.12.2017 17:34, Bill Bejeck wrote:
>
>> Matthias,
>>
>> Overall I agree with what you've presented here.
>>
>> Initially, I was hesitant to remove information from the context of the
>> result records (Joins or Aggregations) with the thought that when there
>> are
>> unexpected results, the source information would be useful for tracing
>> back
>> where the error could have occurred.  But in the case of Joins and
>> Aggregations, the amount of data needed to do meaningful analysis could be
>> too much. For example, a join result could come from two topics so you'd
>> need to keep both original topic names, offsets, etc. (plus the broker
>> could have deleted the records in the interim so even having offset could
>> provide nothing).
>>
>> I'm bit long winded here, but I've come full circle to your original
>> proposal that since Joins and Aggregations produce fundamentally new
>> types,
>> we drop the corresponding information from the context even in the case of
>> single topic aggregations.
>>
>> Thanks,
>> Bill
>>
>> On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>> I agree with Guozhang that just exposing meta data at the source level
>>> might not provide too much value. Furthermore, for timestamps we do
>>> already have a well defined contract and we should exploit it:
>>> timestamps can always be provided in a meaningful way.
>>>
>>> Also, for simple operations like KStream-filter/map the contract is
>>> simple and we can just use it. Same for KTable-filter/map (for new
>>> values).
>>>
>>> For aggregations, join, and oldValue, I could just drop some information
>>> and return `null`/-1, if the result records has no semantically
>>> meaningful meta data.
>>>
>>> For example, for aggregations, we could preserve the partition (as all
>>> agg-input-records have the same partition). For single input topic
>>> aggregation (what I guess is the most prominent case), we can also carry
>>> over the topic name (would be a internal repartitioning topic name
>>> often). Offsets don't have any semantic interpretation IMHO and we could
>>> return -1.
>>>
>>> For joins, we could keep the partition information. Topic and offset are
>>> both unknown/invalid for the output record IMHO.
>>>
>>> For the oldValue case, we can keep partition and for single input topic
>>> case topic name. Timestamp might be -1 for now, but after we added
>>> timestamps to KTable (what we plan to do anyway), we can also return a
>>> valid timestamp. Offset would be -1 again (if we store offset in KTable
>>> too, we could provide all offset as well -- but I don't see too much
>>> value in doing this compared to the storage overhead this implies).
>>>
>>>
>>> WDYT?
>>>
>>>
>>> -Matthias
>>>
>>> On 11/29/17 4:14 AM, Jan Filipiak wrote:
>>>
>>>> Hi,
>>>>
>>>> thank you for the summary and thanks for acknowledging that I do have a
>>>> point here.
>>>>
>>>> I don't like the second Idea at all. Hence I started of this discussion.
>>>>
>>>> I am just disappointed, back then when we had the discussion about how
>>>> to refactor store overload
>>>> and IQ handling, I knew the path we are taking is wrong. Having problems
>>>> implementing these kinda
>>>> features (wich are really simple)  is just a symptom of messed up IQ
>>>> implementation. I wish really bad
>>>> I could have convinced you guys back then. To be honest with IQ we can
>>>> continue here
>>>> as we Materialize but would not send oldValue, but with join you're out
>>>> of luck with current setup.
>>>>
>>>> I of course recommend to do not introduce any optimizations here. Id
>>>> recommend to go towards what
>>>> I recommended already back then. So i would't say we need to optimize
>>>> anything later we need to build
>>>> the topology better in the first place.
>>>>
>>>>
>>>>
>>>>
>>>> On 28.11.2017 21:00, Guozhang Wang wrote:
>>>>
>>>>> Jan,
>>>>>
>>>>> Thanks for your input, I can understand now that the oldValue is also
>>>>> exposed in user customized `filter` function and hence want record
>>>>> context
>>>>> we should expose is a problem. And I think it does brings a good point
>>>>>
>>>> to
>>>
>>>> consider for KIP-159. The discussions maybe a bit confusing to reader
>>>>> though, and hence I'd like to summarize the status quo and with a
>>>>> proposal:
>>>>>
>>>>> In today's Streams DSL, when a KTable is created either from a source
>>>>> topic, or from an stateful operator, we will materialize the KTable
>>>>> with a
>>>>> backing state store; on the other hand, KTables created from a
>>>>> non-stateful
>>>>> operator like filter, will not be backed by a state store by default
>>>>> unless
>>>>> users indicate so (e.g. using the overloaded function with the
>>>>> queryable
>>>>> name or store supplier).
>>>>>
>>>>> For example:
>>>>>
>>>>> KTable table1 = builder.table("topic");
>>>>>
>>>> // a
>>>
>>>> state store created for table1
>>>>> KTable table2 = table1.filter(..);
>>>>> // no state store created for table2
>>>>> KTable table3 = table1.filter(.., "storeName");                  // a
>>>>> state
>>>>> store created for table3
>>>>> KTable table4 = table1.groupBy(..).aggregate(..);            // a
>>>>> state
>>>>> store created for table4
>>>>>
>>>>> Because of that, the filter() operator above on table1 will always be
>>>>> exposed with oldValue and newValue; Damian's point is that, we may
>>>>> optimize
>>>>> the first case such that table1 will only be materialized if users
>>>>> asked so
>>>>> (e.g. using the overloaded function with a store supplier), and in
>>>>> which
>>>>> case, we do not need to pass newValue / oldValue pairs (I think this is
>>>>> what Jan suggests as well, i.e. do filtering before materializing, so
>>>>> that
>>>>> we can have a smaller backed state store as well). But this
>>>>> optimization
>>>>> does not eliminate the possibilities that we may still need to do
>>>>> filter if
>>>>> users does specify "yes I do want to the source KTable itself to be
>>>>> materialized, please". So the concern about how to expose the record
>>>>> context in such cases still persists.
>>>>>
>>>>>
>>>>> With that, regarding to KIP-159 itself, here are my thoughts:
>>>>>
>>>>> 1) if we restrict the scope of exposing record context only to source
>>>>> KTables / KStreams I felt the KIP itself does not bring much value
>>>>> given
>>>>> its required API change because only the SourceKStream can safely
>>>>> maintain
>>>>> its records context, and for SourceKTable if it is materialized, then
>>>>> even
>>>>> non-stateful operators like Join may still have a concern about
>>>>> exposing
>>>>> the record context.
>>>>>
>>>>> 2) an alternative idea is we provide the semantics on how record
>>>>> context
>>>>> would be inherited across the operators for KTable / KStream and
>>>>> expose it
>>>>> in all operators (similarly in PAPI we would expose a much simpler
>>>>> contract), and make it as a public contract that Streams library will
>>>>> guarantee moving forward even we optimize our topology builder; it may
>>>>> not
>>>>> align perfectly with the linear algebraic semantics but practically
>>>>> applicable for most cases; if users semantics do not fit in the
>>>>> provided
>>>>> contract, then they may need to get this themselves (embed such
>>>>> information
>>>>> in the value payload, for example).
>>>>>
>>>>> If people do not like the second idea, I'd suggest we hold on pursuing
>>>>> the
>>>>> first direction since to me its beneficial scope is too limited
>>>>> compared to
>>>>> its cost.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <
>>>>> Jan.Filipiak@trivago.com
>>>>> wrote:
>>>>>
>>>>> Cleary we show the oldValue to the user. We have to, because we filter
>>>>>> after the store.
>>>>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>>>>>> ain/java/org/apache/kafka/streams/kstream/internals/
>>>>>>
>>>>> KTableFilter.java#L96
>>>
>>>>
>>>>>> I cannot help you following this. It is really obvious and I am
>>>>>> running
>>>>>> out of tools for explaining.
>>>>>>
>>>>>> Thanks for understanding my point to put filter before. Not only
>>>>>> would it
>>>>>> make the store smaller. It would make this feature reasonably
>>>>>> possible and
>>>>>> the framework easier. Interestingly it would also help to move IQ
>>>>>> into more
>>>>>> reasonable directions. And it might help understand that we do not
>>>>>> need any
>>>>>> intermediate representation of the topology,
>>>>>>
>>>>>> KIP-182 I have no clue what everyone has with their "bytestores" so
>>>>>> broken. But putting another store after doesn't help when the store
>>>>>> before
>>>>>> is the problem.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>>>>>
>>>>>>    From a DSL point of view, users only see the new value on a
>>>>>>> KTable#filter anyway. So why should it be an issue that we use
>>>>>>> <newValue,oldValue> pair under the hood?
>>>>>>>
>>>>>>> User sees newValue and gets corresponding RecordContext. I can't see
>>>>>>> any
>>>>>>> issue here?
>>>>>>>
>>>>>>> I cannot follow here:
>>>>>>>
>>>>>>> Even when we have a statefull operation last. We move it to the very
>>>>>>>
>>>>>>>> first processor (KtableSource)
>>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>>
>>>>>>>>> With regard to `builder.table().filter()`:
>>>>>>>
>>>>>>> I see you point that it would be good to be able to apply the
>>>>>>> filter()
>>>>>>> first to reduce the stat store size of the table. But how is this
>>>>>>> related to KIP-159?
>>>>>>>
>>>>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>>>>>> putting a custom dummy store into the table and materialize the
>>>>>>> filter
>>>>>>> result afterwards? It's not a nice way to do, but seems to be
>>>>>>>
>>>>>> possible.
>>>
>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>>>>>
>>>>>>> The comment is valid. It falls exactly into this topic, it has
>>>>>>>>
>>>>>>> exactly
>>>
>>>> todo with this!
>>>>>>>> Even when we have a statefull operation last. We move it to the very
>>>>>>>> first processor (KtableSource)
>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>
>>>>>>>> Regarding the other Jiras you are referring to. They harm the
>>>>>>>> project
>>>>>>>> more than they do good!
>>>>>>>> There is no need for this kind of optimizer and meta representation
>>>>>>>> and
>>>>>>>> what not. I hope they
>>>>>>>> never get implemented.
>>>>>>>>
>>>>>>>> Best Jan
>>>>>>>>
>>>>>>>>
>>>>>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>>>>>
>>>>>>>> Jan, i think you comment with respect to filtering is valid, though
>>>>>>>>> not for
>>>>>>>>> this KIP. We have separate JIRAs for topology optimization of
>>>>>>>>> which this
>>>>>>>>> falls into.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Jan,
>>>>>>>>>
>>>>>>>>>> Not sure I understand your argument that "we still going to
>>>>>>>>>> present
>>>>>>>>>> change.oldValue to the filter even though the record context() is
>>>>>>>>>> for
>>>>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
>>>>>>>>>> If yes
>>>>>>>>>> could you point to me which LOC are you concerning about?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> a remark of mine that got missed during migration:
>>>>>>>>>>
>>>>>>>>>>> There is this problem that even though we have
>>>>>>>>>>> source.table.filter.join
>>>>>>>>>>> the state-fullness happens at the table step not a the join
>>>>>>>>>>> step. In a
>>>>>>>>>>> filter
>>>>>>>>>>> we still going to present change.oldValue to the filter even
>>>>>>>>>>>
>>>>>>>>>> though
>>>
>>>> the
>>>>>>>>>>> record context() is for change.newValue. I would go as far as
>>>>>>>>>>> applying
>>>>>>>>>>> the filter before the table processor. Not to just get KIP-159,
>>>>>>>>>>>
>>>>>>>>>> but
>>>
>>>> because
>>>>>>>>>>
>>>>>>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>>>>>>>> filter
>>>>>>>>>>> 99% of my
>>>>>>>>>>> records. my state could be way smaller. Also widely escalates the
>>>>>>>>>>> context
>>>>>>>>>>> of the KIP
>>>>>>>>>>>
>>>>>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>>>>>
>>>>>>>>>>> Best Jan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>>>>>
>>>>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails
>>>>>>>>>>> were
>>>>>>>>>>>
>>>>>>>>>>>> sent
>>>>>>>>>>>> to VOTE thread.
>>>>>>>>>>>>
>>>>>>>>>>>> Copying Guozhang's last summary below. Thanks for this summary.
>>>>>>>>>>>> Very
>>>>>>>>>>>> comprehensive!
>>>>>>>>>>>>
>>>>>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>>>>>> context
>>>>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>>>>>
>>>>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>>>>>>>> carrying the context to more operators than just sources. It
>>>>>>>>>>>> also
>>>>>>>>>>>> seems,
>>>>>>>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>>>>>>>
>>>>>>>>>>>> I still think, that just going with plain (2) is too
>>>>>>>>>>>> restrictive --
>>>>>>>>>>>> but
>>>>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>>>>>
>>>>>>>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>>>>>>>> modify
>>>>>>>>>>>> the record and thus for both, it would be absolutely valid to
>>>>>>>>>>>>
>>>>>>>>>>> keep
>>>
>>>> the
>>>>>>>>>>>> context.
>>>>>>>>>>>>
>>>>>>>>>>>> I personally would keep the context for at least all one-to-one
>>>>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
>>>>>>>>>>>>
>>>>>>>>>>> the
>>>
>>>> context further: at least the offset information is
>>>>>>>>>>>> questionable for
>>>>>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>>>>>> inherited
>>>>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>>>>>> "partition". Thus, I think it's still valuable information we
>>>>>>>>>>>> can
>>>>>>>>>>>> carry
>>>>>>>>>>>> downstreams.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> Jan: which approach are you referring to as "the approach that
>>>>>>>>>>>> is
>>>>>>>>>>>> on the
>>>>>>>>>>>>
>>>>>>>>>>>> table would be perfect"?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>>>>>> exposing the
>>>>>>>>>>>>> record context which has the issues that we have been
>>>>>>>>>>>>> discussing
>>>>>>>>>>>>> right
>>>>>>>>>>>>> now,
>>>>>>>>>>>>> and its semantics is always referring to the "processing
>>>>>>>>>>>>> record" at
>>>>>>>>>>>>>
>>>>>>>>>>>>> hand.
>>>>>>>>>>>>
>>>>>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>>>>
>>>>>>>>>>>> different:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1) the record traversed the topology from source to sink, it
>>>>>>>>>>>>> may be
>>>>>>>>>>>>> transformed into new object or even generate multiple new
>>>>>>>>>>>>>
>>>>>>>>>>>> objects
>>>
>>>> (think:
>>>>>>>>>>>>
>>>>>>>>>>> branch) along the traversal. And the record context is referring
>>>>>>>>>>>
>>>>>>>>>> to
>>>
>>>> this
>>>>>>>>>>>>
>>>>>>>>>>> processing record. Here the "lifetime" of the record lasts for
>>>>>>>>>>> the
>>>>>>>>>>>
>>>>>>>>>>>> entire
>>>>>>>>>>>>
>>>>>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>>>>
>>>>>>>>>>>> treated as
>>>>>>>>>>>>> different transformed values of this record (this applies to
>>>>>>>>>>>>>
>>>>>>>>>>>> join
>>>
>>>> and
>>>>>>>>>>>>> aggregations as well).
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2) the record being processed is wiped out in the first
>>>>>>>>>>>>> operator
>>>>>>>>>>>>> after
>>>>>>>>>>>>> the
>>>>>>>>>>>>> source, and NEW records are forwarded to downstream operators.
>>>>>>>>>>>>> I.e.
>>>>>>>>>>>>>
>>>>>>>>>>>>> each
>>>>>>>>>>>>
>>>>>>>>>>> record only lives between two adjacent operators, once it
>>>>>>>>>>> reached the
>>>>>>>>>>>
>>>>>>>>>>>> new
>>>>>>>>>>>>
>>>>>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>>>>
>>>>>>>>>>>> I think in the past we have talked about Streams under both
>>>>>>>>>>>>> context,
>>>>>>>>>>>>>
>>>>>>>>>>>>> and
>>>>>>>>>>>>
>>>>>>>>>>> we
>>>>>>>>>>>
>>>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>>>>>
>>>>>>>>>>>>> implementation
>>>>>>>>>>>>
>>>>>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>>>>
>>>>>>>>>>>> ends at
>>>>>>>>>>>>> the
>>>>>>>>>>>>> join operator as it is only be materialized, while stream
>>>>>>>>>>>>> record's
>>>>>>>>>>>>> traversal goes through the join operator to further down until
>>>>>>>>>>>>> sinks).
>>>>>>>>>>>>> However if we are going to interpret following 2) above then
>>>>>>>>>>>>>
>>>>>>>>>>>> even
>>>
>>>> for
>>>>>>>>>>>>> non-stateful operators we would not inherit record context.
>>>>>>>>>>>>> What
>>>>>>>>>>>>> we're
>>>>>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>>>>>>>>>>>
>>>>>>>>>>>>> operators,
>>>>>>>>>>>>
>>>>>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>>>>>>
>>>>>>>>>>>> "mapValues"
>>>>>>>>>>>>>       ) and will "end" at many-to-one (stateful) operators
>>>>>>>>>>>>> where NEW
>>>>>>>>>>>>>
>>>>>>>>>>>>> records
>>>>>>>>>>>>
>>>>>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>>>>
>>>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
>>>>>>>>>>>>> the
>>>>>>>>>>>>> same
>>>>>>>>>>>>> page before chatting more.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
>>>>>>>>>>>>> designs
>>>>>>>>>>>>>
>>>>>>>>>>>>> when
>>>>>>>>>>>>
>>>>>>>>>>> punctuate() was not deprecated.
>>>>>>>>>>>
>>>>>>>>>>>> I corrected.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> I just re-read the KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Jeyhun,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>
>>>>>>>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>>>>
>>>>>>>>>>>> moving
>>>>>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>>>>>> conclusion is
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> it would require revamping the public APIs quite a lot, and
>>>>>>>>>>>>>>
>>>>>>>>>>>>> it's
>>>
>>>> not
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> clear
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>>>>>>>>>>>> instead.
>>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
>>>>>>>>>>>>>>> <da...@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         - on-demand commit() feature
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
>>>>>>>>>>>>>>>> simplified
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>
>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
>>>>>>>>>>>>>>>>> agree that
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
>>>>>>>>>>>>>>>>>> close the
>>>>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> best
>>>
>>>> way
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> do.
>>>>>>>>>>>
>>>>>>>>>>>> A
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
>>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>
>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> that
>>>
>>>> flexibility
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
>>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context
>>>>>>>>>>>>>>>>>>> (offset,
>>>>>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>>>>> `commit()`
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>
>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
>>>>>>>>>>>>>>>>>> elaborate a
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
>>>>>>>>>>>>>>>>>> better. Why
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>
>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> explicitly,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>
>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> is an helper object that provide access to record meta
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> data.
>>>
>>>> Requesting
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> specific
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> path.
>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>
>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> internal
>>>
>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
>>>>>>>>>>>>>>>>>>> `RecordContext` is
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> interfaces.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
>>>>>>>>>>>>>>>>>>>>> snippet in
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          public void process(final K1 key, final V1
>>>>>>>>>>>>>>>>>> value) {
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>              recordContext = new RecordContext()
>>>>>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>                  public void commit() {
>>>>>>>>>>>>>>>>>>>>>                      context().commit();
>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>                  public long offset() {
>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> context().recordContext().offs
>>>
>>>> et();
>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>                  public long timestamp() {
>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>                  public String topic() {
>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> context().recordContext().topi
>>>
>>>> c();
>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>>                  public int partition() {
>>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>>            };
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> IMO.
>>>>>>>>>>>
>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
>>>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> amount
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>
>>>>>>>>>>>> methods and it is logical to enable inheritance between
>>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> two.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> below 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 has gone
>>>>>>>>>>>>>>>>>>>>> though
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>
>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> about
>>>
>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>
>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> internal
>>>
>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Streams
>>>
>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>>> ka/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
>>
>>


-- 
-- Guozhang

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

Posted by Jan Filipiak <Ja...@trivago.com>.
Thank you Bill,

I think this is reasonable. Do you have any suggestion
for handling oldValues in cases like

builder.table().filter(RichPredicate).join()

where we process a Change with old and new value and dont have a record 
context for old.

my suggestion would be that instead of

SOURCE -> KTABLESOURCE -> KTABLEFILTER -> JOIN -> SINK

we build

SOURCE  -> KTABLEFILTER ->  KTABLESOURCE -> JOIN -> SINK

We should build a topology like this from the beginning and not have
an optimisation phase afterwards.

Any opinions?

Best Jan




On 05.12.2017 17:34, Bill Bejeck wrote:
> Matthias,
>
> Overall I agree with what you've presented here.
>
> Initially, I was hesitant to remove information from the context of the
> result records (Joins or Aggregations) with the thought that when there are
> unexpected results, the source information would be useful for tracing back
> where the error could have occurred.  But in the case of Joins and
> Aggregations, the amount of data needed to do meaningful analysis could be
> too much. For example, a join result could come from two topics so you'd
> need to keep both original topic names, offsets, etc. (plus the broker
> could have deleted the records in the interim so even having offset could
> provide nothing).
>
> I'm bit long winded here, but I've come full circle to your original
> proposal that since Joins and Aggregations produce fundamentally new types,
> we drop the corresponding information from the context even in the case of
> single topic aggregations.
>
> Thanks,
> Bill
>
> On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> I agree with Guozhang that just exposing meta data at the source level
>> might not provide too much value. Furthermore, for timestamps we do
>> already have a well defined contract and we should exploit it:
>> timestamps can always be provided in a meaningful way.
>>
>> Also, for simple operations like KStream-filter/map the contract is
>> simple and we can just use it. Same for KTable-filter/map (for new values).
>>
>> For aggregations, join, and oldValue, I could just drop some information
>> and return `null`/-1, if the result records has no semantically
>> meaningful meta data.
>>
>> For example, for aggregations, we could preserve the partition (as all
>> agg-input-records have the same partition). For single input topic
>> aggregation (what I guess is the most prominent case), we can also carry
>> over the topic name (would be a internal repartitioning topic name
>> often). Offsets don't have any semantic interpretation IMHO and we could
>> return -1.
>>
>> For joins, we could keep the partition information. Topic and offset are
>> both unknown/invalid for the output record IMHO.
>>
>> For the oldValue case, we can keep partition and for single input topic
>> case topic name. Timestamp might be -1 for now, but after we added
>> timestamps to KTable (what we plan to do anyway), we can also return a
>> valid timestamp. Offset would be -1 again (if we store offset in KTable
>> too, we could provide all offset as well -- but I don't see too much
>> value in doing this compared to the storage overhead this implies).
>>
>>
>> WDYT?
>>
>>
>> -Matthias
>>
>> On 11/29/17 4:14 AM, Jan Filipiak wrote:
>>> Hi,
>>>
>>> thank you for the summary and thanks for acknowledging that I do have a
>>> point here.
>>>
>>> I don't like the second Idea at all. Hence I started of this discussion.
>>>
>>> I am just disappointed, back then when we had the discussion about how
>>> to refactor store overload
>>> and IQ handling, I knew the path we are taking is wrong. Having problems
>>> implementing these kinda
>>> features (wich are really simple)  is just a symptom of messed up IQ
>>> implementation. I wish really bad
>>> I could have convinced you guys back then. To be honest with IQ we can
>>> continue here
>>> as we Materialize but would not send oldValue, but with join you're out
>>> of luck with current setup.
>>>
>>> I of course recommend to do not introduce any optimizations here. Id
>>> recommend to go towards what
>>> I recommended already back then. So i would't say we need to optimize
>>> anything later we need to build
>>> the topology better in the first place.
>>>
>>>
>>>
>>>
>>> On 28.11.2017 21:00, Guozhang Wang wrote:
>>>> Jan,
>>>>
>>>> Thanks for your input, I can understand now that the oldValue is also
>>>> exposed in user customized `filter` function and hence want record
>>>> context
>>>> we should expose is a problem. And I think it does brings a good point
>> to
>>>> consider for KIP-159. The discussions maybe a bit confusing to reader
>>>> though, and hence I'd like to summarize the status quo and with a
>>>> proposal:
>>>>
>>>> In today's Streams DSL, when a KTable is created either from a source
>>>> topic, or from an stateful operator, we will materialize the KTable
>>>> with a
>>>> backing state store; on the other hand, KTables created from a
>>>> non-stateful
>>>> operator like filter, will not be backed by a state store by default
>>>> unless
>>>> users indicate so (e.g. using the overloaded function with the queryable
>>>> name or store supplier).
>>>>
>>>> For example:
>>>>
>>>> KTable table1 = builder.table("topic");
>> // a
>>>> state store created for table1
>>>> KTable table2 = table1.filter(..);
>>>> // no state store created for table2
>>>> KTable table3 = table1.filter(.., "storeName");                  // a
>>>> state
>>>> store created for table3
>>>> KTable table4 = table1.groupBy(..).aggregate(..);            // a state
>>>> store created for table4
>>>>
>>>> Because of that, the filter() operator above on table1 will always be
>>>> exposed with oldValue and newValue; Damian's point is that, we may
>>>> optimize
>>>> the first case such that table1 will only be materialized if users
>>>> asked so
>>>> (e.g. using the overloaded function with a store supplier), and in which
>>>> case, we do not need to pass newValue / oldValue pairs (I think this is
>>>> what Jan suggests as well, i.e. do filtering before materializing, so
>>>> that
>>>> we can have a smaller backed state store as well). But this optimization
>>>> does not eliminate the possibilities that we may still need to do
>>>> filter if
>>>> users does specify "yes I do want to the source KTable itself to be
>>>> materialized, please". So the concern about how to expose the record
>>>> context in such cases still persists.
>>>>
>>>>
>>>> With that, regarding to KIP-159 itself, here are my thoughts:
>>>>
>>>> 1) if we restrict the scope of exposing record context only to source
>>>> KTables / KStreams I felt the KIP itself does not bring much value given
>>>> its required API change because only the SourceKStream can safely
>>>> maintain
>>>> its records context, and for SourceKTable if it is materialized, then
>>>> even
>>>> non-stateful operators like Join may still have a concern about exposing
>>>> the record context.
>>>>
>>>> 2) an alternative idea is we provide the semantics on how record context
>>>> would be inherited across the operators for KTable / KStream and
>>>> expose it
>>>> in all operators (similarly in PAPI we would expose a much simpler
>>>> contract), and make it as a public contract that Streams library will
>>>> guarantee moving forward even we optimize our topology builder; it may
>>>> not
>>>> align perfectly with the linear algebraic semantics but practically
>>>> applicable for most cases; if users semantics do not fit in the provided
>>>> contract, then they may need to get this themselves (embed such
>>>> information
>>>> in the value payload, for example).
>>>>
>>>> If people do not like the second idea, I'd suggest we hold on pursuing
>>>> the
>>>> first direction since to me its beneficial scope is too limited
>>>> compared to
>>>> its cost.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <Jan.Filipiak@trivago.com
>>>> wrote:
>>>>
>>>>> Cleary we show the oldValue to the user. We have to, because we filter
>>>>> after the store.
>>>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>>>>> ain/java/org/apache/kafka/streams/kstream/internals/
>> KTableFilter.java#L96
>>>>>
>>>>> I cannot help you following this. It is really obvious and I am running
>>>>> out of tools for explaining.
>>>>>
>>>>> Thanks for understanding my point to put filter before. Not only
>>>>> would it
>>>>> make the store smaller. It would make this feature reasonably
>>>>> possible and
>>>>> the framework easier. Interestingly it would also help to move IQ
>>>>> into more
>>>>> reasonable directions. And it might help understand that we do not
>>>>> need any
>>>>> intermediate representation of the topology,
>>>>>
>>>>> KIP-182 I have no clue what everyone has with their "bytestores" so
>>>>> broken. But putting another store after doesn't help when the store
>>>>> before
>>>>> is the problem.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>>>>
>>>>>>    From a DSL point of view, users only see the new value on a
>>>>>> KTable#filter anyway. So why should it be an issue that we use
>>>>>> <newValue,oldValue> pair under the hood?
>>>>>>
>>>>>> User sees newValue and gets corresponding RecordContext. I can't see
>>>>>> any
>>>>>> issue here?
>>>>>>
>>>>>> I cannot follow here:
>>>>>>
>>>>>> Even when we have a statefull operation last. We move it to the very
>>>>>>>> first processor (KtableSource)
>>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>>
>>>>>> With regard to `builder.table().filter()`:
>>>>>>
>>>>>> I see you point that it would be good to be able to apply the filter()
>>>>>> first to reduce the stat store size of the table. But how is this
>>>>>> related to KIP-159?
>>>>>>
>>>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>>>>> putting a custom dummy store into the table and materialize the filter
>>>>>> result afterwards? It's not a nice way to do, but seems to be
>> possible.
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>>>>
>>>>>>> The comment is valid. It falls exactly into this topic, it has
>> exactly
>>>>>>> todo with this!
>>>>>>> Even when we have a statefull operation last. We move it to the very
>>>>>>> first processor (KtableSource)
>>>>>>> and therefore cant present a proper RecordContext.
>>>>>>>
>>>>>>> Regarding the other Jiras you are referring to. They harm the project
>>>>>>> more than they do good!
>>>>>>> There is no need for this kind of optimizer and meta representation
>>>>>>> and
>>>>>>> what not. I hope they
>>>>>>> never get implemented.
>>>>>>>
>>>>>>> Best Jan
>>>>>>>
>>>>>>>
>>>>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>>>>
>>>>>>>> Jan, i think you comment with respect to filtering is valid, though
>>>>>>>> not for
>>>>>>>> this KIP. We have separate JIRAs for topology optimization of
>>>>>>>> which this
>>>>>>>> falls into.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Jan,
>>>>>>>>> Not sure I understand your argument that "we still going to present
>>>>>>>>> change.oldValue to the filter even though the record context() is
>>>>>>>>> for
>>>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
>>>>>>>>> If yes
>>>>>>>>> could you point to me which LOC are you concerning about?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> a remark of mine that got missed during migration:
>>>>>>>>>> There is this problem that even though we have
>>>>>>>>>> source.table.filter.join
>>>>>>>>>> the state-fullness happens at the table step not a the join
>>>>>>>>>> step. In a
>>>>>>>>>> filter
>>>>>>>>>> we still going to present change.oldValue to the filter even
>> though
>>>>>>>>>> the
>>>>>>>>>> record context() is for change.newValue. I would go as far as
>>>>>>>>>> applying
>>>>>>>>>> the filter before the table processor. Not to just get KIP-159,
>> but
>>>>>>>>> because
>>>>>>>>>
>>>>>>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>>>>>>> filter
>>>>>>>>>> 99% of my
>>>>>>>>>> records. my state could be way smaller. Also widely escalates the
>>>>>>>>>> context
>>>>>>>>>> of the KIP
>>>>>>>>>>
>>>>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>>>>
>>>>>>>>>> Best Jan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>>>>
>>>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>>>>>>>> sent
>>>>>>>>>>> to VOTE thread.
>>>>>>>>>>>
>>>>>>>>>>> Copying Guozhang's last summary below. Thanks for this summary.
>>>>>>>>>>> Very
>>>>>>>>>>> comprehensive!
>>>>>>>>>>>
>>>>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>>>>> context
>>>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>>>>
>>>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>>>>>>> carrying the context to more operators than just sources. It also
>>>>>>>>>>> seems,
>>>>>>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>>>>>>
>>>>>>>>>>> I still think, that just going with plain (2) is too
>>>>>>>>>>> restrictive --
>>>>>>>>>>> but
>>>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>>>>
>>>>>>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>>>>>>> modify
>>>>>>>>>>> the record and thus for both, it would be absolutely valid to
>> keep
>>>>>>>>>>> the
>>>>>>>>>>> context.
>>>>>>>>>>>
>>>>>>>>>>> I personally would keep the context for at least all one-to-one
>>>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
>> the
>>>>>>>>>>> context further: at least the offset information is
>>>>>>>>>>> questionable for
>>>>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>>>>> inherited
>>>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>>>>> "partition". Thus, I think it's still valuable information we can
>>>>>>>>>>> carry
>>>>>>>>>>> downstreams.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> Jan: which approach are you referring to as "the approach that is
>>>>>>>>>>> on the
>>>>>>>>>>>
>>>>>>>>>>>> table would be perfect"?
>>>>>>>>>>>>
>>>>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>>>>> exposing the
>>>>>>>>>>>> record context which has the issues that we have been discussing
>>>>>>>>>>>> right
>>>>>>>>>>>> now,
>>>>>>>>>>>> and its semantics is always referring to the "processing
>>>>>>>>>>>> record" at
>>>>>>>>>>>>
>>>>>>>>>>> hand.
>>>>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>>>>> different:
>>>>>>>>>>>>
>>>>>>>>>>>> 1) the record traversed the topology from source to sink, it
>>>>>>>>>>>> may be
>>>>>>>>>>>> transformed into new object or even generate multiple new
>> objects
>>>>>>>>>>> (think:
>>>>>>>>>> branch) along the traversal. And the record context is referring
>> to
>>>>>>>>>>> this
>>>>>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>>>>>>>>>> entire
>>>>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>>>>> treated as
>>>>>>>>>>>> different transformed values of this record (this applies to
>> join
>>>>>>>>>>>> and
>>>>>>>>>>>> aggregations as well).
>>>>>>>>>>>>
>>>>>>>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>>>>>>>> after
>>>>>>>>>>>> the
>>>>>>>>>>>> source, and NEW records are forwarded to downstream operators.
>>>>>>>>>>>> I.e.
>>>>>>>>>>>>
>>>>>>>>>>> each
>>>>>>>>>> record only lives between two adjacent operators, once it
>>>>>>>>>> reached the
>>>>>>>>>>> new
>>>>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>>>>> I think in the past we have talked about Streams under both
>>>>>>>>>>>> context,
>>>>>>>>>>>>
>>>>>>>>>>> and
>>>>>>>>>> we
>>>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>>>>
>>>>>>>>>>> implementation
>>>>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>>>>> ends at
>>>>>>>>>>>> the
>>>>>>>>>>>> join operator as it is only be materialized, while stream
>>>>>>>>>>>> record's
>>>>>>>>>>>> traversal goes through the join operator to further down until
>>>>>>>>>>>> sinks).
>>>>>>>>>>>> However if we are going to interpret following 2) above then
>> even
>>>>>>>>>>>> for
>>>>>>>>>>>> non-stateful operators we would not inherit record context. What
>>>>>>>>>>>> we're
>>>>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>>>>
>>>>>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>>>>>>>>>>
>>>>>>>>>>> operators,
>>>>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>>>>>>> "mapValues"
>>>>>>>>>>>>       ) and will "end" at many-to-one (stateful) operators
>>>>>>>>>>>> where NEW
>>>>>>>>>>>>
>>>>>>>>>>> records
>>>>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
>>>>>>>>>>>> the
>>>>>>>>>>>> same
>>>>>>>>>>>> page before chatting more.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
>>>>>>>>>>>> designs
>>>>>>>>>>>>
>>>>>>>>>>> when
>>>>>>>>>> punctuate() was not deprecated.
>>>>>>>>>>>> I corrected.
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> I just re-read the KIP.
>>>>>>>>>>>>
>>>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>>>>> methods.
>>>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>>>>>>> instead
>>>>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Jeyhun,
>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>
>>>>>>>>>>>>> think
>>>>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>>>>> conclusion is
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> that
>>>>>>>>>>>>> it would require revamping the public APIs quite a lot, and
>> it's
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> clear
>>>>>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>>>>>>>>>>> instead.
>>>>>>>>>> Guozhang
>>>>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
>>>>>>>>>>>>>> <da...@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         - on-demand commit() feature
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
>>>>>>>>>>>>>>> simplified
>>>>>>>>>>>>>>> the
>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
>>>>>>>>>>>>>>>> agree that
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
>>>>>>>>>>>>>>>>> close the
>>>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
>> best
>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to
>>>>>>>>>> do.
>>>>>>>>>>>>>>>> A
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> an
>>>>>>>>>> argument
>>>>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
>> that
>>>>>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>>>> `commit()`
>>>>>>>>>> at
>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
>>>>>>>>>>>>>>>>> elaborate a
>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
>>>>>>>>>>>>>>>>> better. Why
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> would
>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> explicitly,
>>>>>>>>>> ie,
>>>>>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>> is an helper object that provide access to record meta
>> data.
>>>>>>>>>>>>>>>>>> Requesting
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>>>>> specific
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> this
>>>>>>>>>> path.
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>> enforce
>>>>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>> internal
>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
>>>>>>>>>>>>>>>>>> `RecordContext` is
>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>>>> interfaces.
>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
>>>>>>>>>>>>>>>>>>>> snippet in
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>>>          public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>>>>>>>              recordContext = new RecordContext()
>>>>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>                  public void commit() {
>>>>>>>>>>>>>>>>>>>>                      context().commit();
>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>                  public long offset() {
>>>>>>>>>>>>>>>>>>>>                      return
>> context().recordContext().offs
>>>>>>>>>>>>>>>>>>>> et();
>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>                  public long timestamp() {
>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>                  public String topic() {
>>>>>>>>>>>>>>>>>>>>                      return
>> context().recordContext().topi
>>>>>>>>>>>>>>>>>>>> c();
>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>                  @Override
>>>>>>>>>>>>>>>>>>>>                  public int partition() {
>>>>>>>>>>>>>>>>>>>>                      return
>>>>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>>>>                  }
>>>>>>>>>>>>>>>>>>>>            };
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>> IMO.
>>>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
>>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> amount
>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> methods and it is logical to enable inheritance between
>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>> two.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> below 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 has gone
>>>>>>>>>>>>>>>>>>>> though
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
>> about
>>>>>>>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>> enforce
>>>>>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
>> internal
>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
>> Streams
>>>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>>>> ka/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. Sax <
>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 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 DSL users should not
>> worry
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert
>> a
>>>>>>>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>>>          VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>>>>                   final K key,
>>>>>>>>>>>>>>>>>>>>>>>>>                   final RecordContext
>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>>>          VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>>>>                   final K key,
>>>>>>>>>>>>>>>>>>>>>>>>>                   final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>>>>>>>                   final ProcessorContext
>>>>>>>>>>>>>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/
>> jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>          VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>>>> pairing
>>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                                         final
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>>>                                         final
>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K,
>>>>>>>>>>>>>>>>>>>>>>>>>> VT>
>>>>>>>>>>>>>>>> table,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>                                       final
>>>>>>>>>>>>>>>>>>>>>>>>>>> RichValueJoiner<?
>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                                       final
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<K>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                                       final
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding
>>>>>>>>>>>>>>>>>>>>>>>>>>> three APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>> a
>>>
>>



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

Posted by Bill Bejeck <bb...@gmail.com>.
Matthias,

Overall I agree with what you've presented here.

Initially, I was hesitant to remove information from the context of the
result records (Joins or Aggregations) with the thought that when there are
unexpected results, the source information would be useful for tracing back
where the error could have occurred.  But in the case of Joins and
Aggregations, the amount of data needed to do meaningful analysis could be
too much. For example, a join result could come from two topics so you'd
need to keep both original topic names, offsets, etc. (plus the broker
could have deleted the records in the interim so even having offset could
provide nothing).

I'm bit long winded here, but I've come full circle to your original
proposal that since Joins and Aggregations produce fundamentally new types,
we drop the corresponding information from the context even in the case of
single topic aggregations.

Thanks,
Bill

On Mon, Dec 4, 2017 at 7:02 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> I agree with Guozhang that just exposing meta data at the source level
> might not provide too much value. Furthermore, for timestamps we do
> already have a well defined contract and we should exploit it:
> timestamps can always be provided in a meaningful way.
>
> Also, for simple operations like KStream-filter/map the contract is
> simple and we can just use it. Same for KTable-filter/map (for new values).
>
> For aggregations, join, and oldValue, I could just drop some information
> and return `null`/-1, if the result records has no semantically
> meaningful meta data.
>
> For example, for aggregations, we could preserve the partition (as all
> agg-input-records have the same partition). For single input topic
> aggregation (what I guess is the most prominent case), we can also carry
> over the topic name (would be a internal repartitioning topic name
> often). Offsets don't have any semantic interpretation IMHO and we could
> return -1.
>
> For joins, we could keep the partition information. Topic and offset are
> both unknown/invalid for the output record IMHO.
>
> For the oldValue case, we can keep partition and for single input topic
> case topic name. Timestamp might be -1 for now, but after we added
> timestamps to KTable (what we plan to do anyway), we can also return a
> valid timestamp. Offset would be -1 again (if we store offset in KTable
> too, we could provide all offset as well -- but I don't see too much
> value in doing this compared to the storage overhead this implies).
>
>
> WDYT?
>
>
> -Matthias
>
> On 11/29/17 4:14 AM, Jan Filipiak wrote:
> > Hi,
> >
> > thank you for the summary and thanks for acknowledging that I do have a
> > point here.
> >
> > I don't like the second Idea at all. Hence I started of this discussion.
> >
> > I am just disappointed, back then when we had the discussion about how
> > to refactor store overload
> > and IQ handling, I knew the path we are taking is wrong. Having problems
> > implementing these kinda
> > features (wich are really simple)  is just a symptom of messed up IQ
> > implementation. I wish really bad
> > I could have convinced you guys back then. To be honest with IQ we can
> > continue here
> > as we Materialize but would not send oldValue, but with join you're out
> > of luck with current setup.
> >
> > I of course recommend to do not introduce any optimizations here. Id
> > recommend to go towards what
> > I recommended already back then. So i would't say we need to optimize
> > anything later we need to build
> > the topology better in the first place.
> >
> >
> >
> >
> > On 28.11.2017 21:00, Guozhang Wang wrote:
> >> Jan,
> >>
> >> Thanks for your input, I can understand now that the oldValue is also
> >> exposed in user customized `filter` function and hence want record
> >> context
> >> we should expose is a problem. And I think it does brings a good point
> to
> >> consider for KIP-159. The discussions maybe a bit confusing to reader
> >> though, and hence I'd like to summarize the status quo and with a
> >> proposal:
> >>
> >> In today's Streams DSL, when a KTable is created either from a source
> >> topic, or from an stateful operator, we will materialize the KTable
> >> with a
> >> backing state store; on the other hand, KTables created from a
> >> non-stateful
> >> operator like filter, will not be backed by a state store by default
> >> unless
> >> users indicate so (e.g. using the overloaded function with the queryable
> >> name or store supplier).
> >>
> >> For example:
> >>
> >> KTable table1 = builder.table("topic");
> // a
> >> state store created for table1
> >> KTable table2 = table1.filter(..);
> >> // no state store created for table2
> >> KTable table3 = table1.filter(.., "storeName");                  // a
> >> state
> >> store created for table3
> >> KTable table4 = table1.groupBy(..).aggregate(..);            // a state
> >> store created for table4
> >>
> >> Because of that, the filter() operator above on table1 will always be
> >> exposed with oldValue and newValue; Damian's point is that, we may
> >> optimize
> >> the first case such that table1 will only be materialized if users
> >> asked so
> >> (e.g. using the overloaded function with a store supplier), and in which
> >> case, we do not need to pass newValue / oldValue pairs (I think this is
> >> what Jan suggests as well, i.e. do filtering before materializing, so
> >> that
> >> we can have a smaller backed state store as well). But this optimization
> >> does not eliminate the possibilities that we may still need to do
> >> filter if
> >> users does specify "yes I do want to the source KTable itself to be
> >> materialized, please". So the concern about how to expose the record
> >> context in such cases still persists.
> >>
> >>
> >> With that, regarding to KIP-159 itself, here are my thoughts:
> >>
> >> 1) if we restrict the scope of exposing record context only to source
> >> KTables / KStreams I felt the KIP itself does not bring much value given
> >> its required API change because only the SourceKStream can safely
> >> maintain
> >> its records context, and for SourceKTable if it is materialized, then
> >> even
> >> non-stateful operators like Join may still have a concern about exposing
> >> the record context.
> >>
> >> 2) an alternative idea is we provide the semantics on how record context
> >> would be inherited across the operators for KTable / KStream and
> >> expose it
> >> in all operators (similarly in PAPI we would expose a much simpler
> >> contract), and make it as a public contract that Streams library will
> >> guarantee moving forward even we optimize our topology builder; it may
> >> not
> >> align perfectly with the linear algebraic semantics but practically
> >> applicable for most cases; if users semantics do not fit in the provided
> >> contract, then they may need to get this themselves (embed such
> >> information
> >> in the value payload, for example).
> >>
> >> If people do not like the second idea, I'd suggest we hold on pursuing
> >> the
> >> first direction since to me its beneficial scope is too limited
> >> compared to
> >> its cost.
> >>
> >>
> >> Guozhang
> >>
> >>
> >>
> >> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <Jan.Filipiak@trivago.com
> >
> >> wrote:
> >>
> >>> Cleary we show the oldValue to the user. We have to, because we filter
> >>> after the store.
> >>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
> >>> ain/java/org/apache/kafka/streams/kstream/internals/
> KTableFilter.java#L96
> >>>
> >>>
> >>> I cannot help you following this. It is really obvious and I am running
> >>> out of tools for explaining.
> >>>
> >>> Thanks for understanding my point to put filter before. Not only
> >>> would it
> >>> make the store smaller. It would make this feature reasonably
> >>> possible and
> >>> the framework easier. Interestingly it would also help to move IQ
> >>> into more
> >>> reasonable directions. And it might help understand that we do not
> >>> need any
> >>> intermediate representation of the topology,
> >>>
> >>> KIP-182 I have no clue what everyone has with their "bytestores" so
> >>> broken. But putting another store after doesn't help when the store
> >>> before
> >>> is the problem.
> >>>
> >>>
> >>>
> >>>
> >>> On 24.11.2017 05:08, Matthias J. Sax wrote:
> >>>
> >>>>   From a DSL point of view, users only see the new value on a
> >>>> KTable#filter anyway. So why should it be an issue that we use
> >>>> <newValue,oldValue> pair under the hood?
> >>>>
> >>>> User sees newValue and gets corresponding RecordContext. I can't see
> >>>> any
> >>>> issue here?
> >>>>
> >>>> I cannot follow here:
> >>>>
> >>>> Even when we have a statefull operation last. We move it to the very
> >>>>>> first processor (KtableSource)
> >>>>>> and therefore cant present a proper RecordContext.
> >>>>>>
> >>>> With regard to `builder.table().filter()`:
> >>>>
> >>>> I see you point that it would be good to be able to apply the filter()
> >>>> first to reduce the stat store size of the table. But how is this
> >>>> related to KIP-159?
> >>>>
> >>>> Btw: with KIP-182, I am wondering if this would not be possible, by
> >>>> putting a custom dummy store into the table and materialize the filter
> >>>> result afterwards? It's not a nice way to do, but seems to be
> possible.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
> >>>>
> >>>>> The comment is valid. It falls exactly into this topic, it has
> exactly
> >>>>> todo with this!
> >>>>> Even when we have a statefull operation last. We move it to the very
> >>>>> first processor (KtableSource)
> >>>>> and therefore cant present a proper RecordContext.
> >>>>>
> >>>>> Regarding the other Jiras you are referring to. They harm the project
> >>>>> more than they do good!
> >>>>> There is no need for this kind of optimizer and meta representation
> >>>>> and
> >>>>> what not. I hope they
> >>>>> never get implemented.
> >>>>>
> >>>>> Best Jan
> >>>>>
> >>>>>
> >>>>> On 22.11.2017 14:44, Damian Guy wrote:
> >>>>>
> >>>>>> Jan, i think you comment with respect to filtering is valid, though
> >>>>>> not for
> >>>>>> this KIP. We have separate JIRAs for topology optimization of
> >>>>>> which this
> >>>>>> falls into.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Damian
> >>>>>>
> >>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Jan,
> >>>>>>> Not sure I understand your argument that "we still going to present
> >>>>>>> change.oldValue to the filter even though the record context() is
> >>>>>>> for
> >>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
> >>>>>>> If yes
> >>>>>>> could you point to me which LOC are you concerning about?
> >>>>>>>
> >>>>>>>
> >>>>>>> Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
> >>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>> a remark of mine that got missed during migration:
> >>>>>>>> There is this problem that even though we have
> >>>>>>>> source.table.filter.join
> >>>>>>>> the state-fullness happens at the table step not a the join
> >>>>>>>> step. In a
> >>>>>>>> filter
> >>>>>>>> we still going to present change.oldValue to the filter even
> though
> >>>>>>>> the
> >>>>>>>> record context() is for change.newValue. I would go as far as
> >>>>>>>> applying
> >>>>>>>> the filter before the table processor. Not to just get KIP-159,
> but
> >>>>>>>>
> >>>>>>> because
> >>>>>>>
> >>>>>>>> I think its a side effect of a non ideal topology layout. If i can
> >>>>>>>> filter
> >>>>>>>> 99% of my
> >>>>>>>> records. my state could be way smaller. Also widely escalates the
> >>>>>>>> context
> >>>>>>>> of the KIP
> >>>>>>>>
> >>>>>>>> I can only see upsides of executing the filter first.
> >>>>>>>>
> >>>>>>>> Best Jan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
> >>>>>>>>
> >>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
> >>>>>>>>> sent
> >>>>>>>>> to VOTE thread.
> >>>>>>>>>
> >>>>>>>>> Copying Guozhang's last summary below. Thanks for this summary.
> >>>>>>>>> Very
> >>>>>>>>> comprehensive!
> >>>>>>>>>
> >>>>>>>>> It seems, we all agree, that the current implementation of the
> >>>>>>>>> context
> >>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
> >>>>>>>>>
> >>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
> >>>>>>>>> carrying the context to more operators than just sources. It also
> >>>>>>>>> seems,
> >>>>>>>>> that we all agree, that many-to-one operations void the context.
> >>>>>>>>>
> >>>>>>>>> I still think, that just going with plain (2) is too
> >>>>>>>>> restrictive --
> >>>>>>>>> but
> >>>>>>>>> I am also fine if we don't go with the full proposal of (3).
> >>>>>>>>>
> >>>>>>>>> Also note, that the two operators filter() and filterNot() don't
> >>>>>>>>> modify
> >>>>>>>>> the record and thus for both, it would be absolutely valid to
> keep
> >>>>>>>>> the
> >>>>>>>>> context.
> >>>>>>>>>
> >>>>>>>>> I personally would keep the context for at least all one-to-one
> >>>>>>>>> operators. One-to-many is debatable and I am fine to not carry
> the
> >>>>>>>>> context further: at least the offset information is
> >>>>>>>>> questionable for
> >>>>>>>>> this case -- note thought, that semantically, the timestamp is
> >>>>>>>>> inherited
> >>>>>>>>> via one-to-many, and I also think this applies to "topic" and
> >>>>>>>>> "partition". Thus, I think it's still valuable information we can
> >>>>>>>>> carry
> >>>>>>>>> downstreams.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>> Jan: which approach are you referring to as "the approach that is
> >>>>>>>>> on the
> >>>>>>>>>
> >>>>>>>>>> table would be perfect"?
> >>>>>>>>>>
> >>>>>>>>>> Note that in today's PAPI layer we are already effectively
> >>>>>>>>>> exposing the
> >>>>>>>>>> record context which has the issues that we have been discussing
> >>>>>>>>>> right
> >>>>>>>>>> now,
> >>>>>>>>>> and its semantics is always referring to the "processing
> >>>>>>>>>> record" at
> >>>>>>>>>>
> >>>>>>>>> hand.
> >>>>>>>> More specifically, we can think of processing a record a bit
> >>>>>>>>>> different:
> >>>>>>>>>>
> >>>>>>>>>> 1) the record traversed the topology from source to sink, it
> >>>>>>>>>> may be
> >>>>>>>>>> transformed into new object or even generate multiple new
> objects
> >>>>>>>>>>
> >>>>>>>>> (think:
> >>>>>>>> branch) along the traversal. And the record context is referring
> to
> >>>>>>>>> this
> >>>>>>>> processing record. Here the "lifetime" of the record lasts for the
> >>>>>>>>> entire
> >>>>>>>> topology traversal and any new records of this traversal is
> >>>>>>>>>> treated as
> >>>>>>>>>> different transformed values of this record (this applies to
> join
> >>>>>>>>>> and
> >>>>>>>>>> aggregations as well).
> >>>>>>>>>>
> >>>>>>>>>> 2) the record being processed is wiped out in the first operator
> >>>>>>>>>> after
> >>>>>>>>>> the
> >>>>>>>>>> source, and NEW records are forwarded to downstream operators.
> >>>>>>>>>> I.e.
> >>>>>>>>>>
> >>>>>>>>> each
> >>>>>>>> record only lives between two adjacent operators, once it
> >>>>>>>> reached the
> >>>>>>>>> new
> >>>>>>>> operator it's lifetime has ended and new records are generated.
> >>>>>>>>>> I think in the past we have talked about Streams under both
> >>>>>>>>>> context,
> >>>>>>>>>>
> >>>>>>>>> and
> >>>>>>>> we
> >>>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
> >>>>>>>>>> understandable for users as it does not leak any internal
> >>>>>>>>>>
> >>>>>>>>> implementation
> >>>>>>>> details (e.g. for stream-table joins, table record's traversal
> >>>>>>>>>> ends at
> >>>>>>>>>> the
> >>>>>>>>>> join operator as it is only be materialized, while stream
> >>>>>>>>>> record's
> >>>>>>>>>> traversal goes through the join operator to further down until
> >>>>>>>>>> sinks).
> >>>>>>>>>> However if we are going to interpret following 2) above then
> even
> >>>>>>>>>> for
> >>>>>>>>>> non-stateful operators we would not inherit record context. What
> >>>>>>>>>> we're
> >>>>>>>>>> discussing now, seems to infer a third semantics:
> >>>>>>>>>>
> >>>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
> >>>>>>>>>>
> >>>>>>>>> operators,
> >>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
> >>>>>>>>>> "mapValues"
> >>>>>>>>>>      ) and will "end" at many-to-one (stateful) operators
> >>>>>>>>>> where NEW
> >>>>>>>>>>
> >>>>>>>>> records
> >>>>>>>> will be generated and forwarded to the downstream operators.
> >>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
> >>>>>>>>>> the
> >>>>>>>>>> same
> >>>>>>>>>> page before chatting more.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Guozhang
> >>>>>>>>>>
> >>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
> >>>>>>>>> Hi Matthias,
> >>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
> >>>>>>>>>> designs
> >>>>>>>>>>
> >>>>>>>>> when
> >>>>>>>> punctuate() was not deprecated.
> >>>>>>>>>> I corrected.
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Jeyhun
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
> >>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> I just re-read the KIP.
> >>>>>>>>>>
> >>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
> >>>>>>>>>>> methods.
> >>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
> >>>>>>>>>>> instead
> >>>>>>>>>>> of introducing it as deprecated.
> >>>>>>>>>>>
> >>>>>>>>>>> Otherwise looks good to me.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for being so patient!
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Jeyhun,
> >>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
> >>>>>>>>>>>> should
> >>>>>>>>>>>>
> >>>>>>>>>>> think
> >>>>>>>> carefully if we should add this functionality to the DSL layer
> >>>>>>>>>>>> moving
> >>>>>>>>>>>> forward since from what we discovered working on it the
> >>>>>>>>>>>> conclusion is
> >>>>>>>>>>>>
> >>>>>>>>>>>> that
> >>>>>>>>>>> it would require revamping the public APIs quite a lot, and
> it's
> >>>>>>>>>>>> not
> >>>>>>>>>>>>
> >>>>>>>>>>>> clear
> >>>>>>>>>>> if it is a good trade-off than asking users to call process()
> >>>>>>>>>>> instead.
> >>>>>>>> Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
> >>>>>>>>>>>> <da...@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Do we need to remove the line that says:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        - on-demand commit() feature
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
> >>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>> Hi,
> >>>>>>>>>>>>
> >>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
> >>>>>>>>>>>>> simplified
> >>>>>>>>>>>>> the
> >>>>>>>> overall design of KIP a lot.
> >>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
> >>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
> >>>>>>>>>>>>>> agree that
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> but also we need a commit() method
> >>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
> >>>>>>>>>>>>>>> close the
> >>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Matthias,
> >>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the
> best
> >>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> to
> >>>>>>>> do.
> >>>>>>>>>>>>>> A
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> bit of history behind this design.
> >>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
> >>>>>>>>>>>>>>>> itself
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> an
> >>>>>>>> argument
> >>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users
> that
> >>>>>>>>>>>>>>>> flexibility
> >>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
> >>>>>>>>>>>>>>> level
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> information and not Record level info. The only thing we
> >>>>>>>>>>>>>>>> need ij
> >>>>>>>>>>>>>>>> ProcessorContext is commit() method.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
> >>>>>>>>>>>>>>>> timestamp
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
> >>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> ProcessorContext.commit()
> >>>>>>>>>>>>>>> ).
> >>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ProcessorContext() .
> >>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> KIP/included
> >>>>>>>>>>>>>> jiras please let me know.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
> >>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>> I am personally still not convinced, that we should add
> >>>>>>>>>>>>>>> `commit()`
> >>>>>>>> at
> >>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
> >>>>>>>>>>>>>>> elaborate a
> >>>>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
> >>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> exposed
> >>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
> >>>>>>>>>>>>>>> better. Why
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> would
> >>>>>>>> anybody that uses the DSL ever want to request a commit? To
> >>>>>>>>>>>>>>>>> me,
> >>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> explicitly,
> >>>>>>>> ie,
> >>>>>>>>>>>>>>> via Processor API.
> >>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
> >>>>>>>>>>>>>>>>> that we
> >>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `RecordContext`
> >>>>>>>>>>>>>>> is an helper object that provide access to record meta
> data.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Requesting
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
> >>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
> >>>>>>>>>>>>>> specific
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> record.
> >>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
> >>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> this
> >>>>>>>> path.
> >>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>> Thanks for your suggestions.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> misunderstanding.
> >>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
> >>>>>>>>>>>>>>>>>> ProcessorContext,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> to
> >>>>>>>> enforce
> >>>>>>>>>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
> internal
> >>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
> >>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> changed
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> this call.
> >>>>>>>>>>>>>>>>> - I think we should not deprecate
> >>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
> >>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> main
> >>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
> >>>>>>>>>>>>>>>> `RecordContext` is
> >>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
> >>>>>>>>>>>>>>>>> interfaces.
> >>>>>>>> So
> >>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
> >>>>>>>>>>>>>>>>>> `RecordContext`
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
> >>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
> >>>>>>>>>>>>>>>>>> snippet in
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> KIP-159):
> >>>>>>>>>>>>>>>> @Override
> >>>>>>>>>>>>>>>         public void process(final K1 key, final V1 value) {
> >>>>>>>>>>>>>>>>>>             recordContext = new RecordContext()
> >>>>>>>>>>>>>>>>>> {               //
> >>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
> >>>>>>>>>>>>>>>>>>                 @Override
> >>>>>>>>>>>>>>>>>>                 public void commit() {
> >>>>>>>>>>>>>>>>>>                     context().commit();
> >>>>>>>>>>>>>>>>>>                 }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>                 @Override
> >>>>>>>>>>>>>>>>>>                 public long offset() {
> >>>>>>>>>>>>>>>>>>                     return
> context().recordContext().offs
> >>>>>>>>>>>>>>>>>> et();
> >>>>>>>>>>>>>>>>>>                 }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>                 @Override
> >>>>>>>>>>>>>>>>>>                 public long timestamp() {
> >>>>>>>>>>>>>>>>>>                     return
> >>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
> >>>>>>>>>>>>>>>>>>                 }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>                 @Override
> >>>>>>>>>>>>>>>>>>                 public String topic() {
> >>>>>>>>>>>>>>>>>>                     return
> context().recordContext().topi
> >>>>>>>>>>>>>>>>>> c();
> >>>>>>>>>>>>>>>>>>                 }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>                 @Override
> >>>>>>>>>>>>>>>>>>                 public int partition() {
> >>>>>>>>>>>>>>>>>>                     return
> >>>>>>>>>>>>>>>>>> context().recordContext().partition();
> >>>>>>>>>>>>>>>>>>                 }
> >>>>>>>>>>>>>>>>>>           };
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> case
> >>>>>>>> IMO.
> >>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
> >>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
> >>>>>>>>>>>>>>>>>> `RecordContext()`
> >>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
> >>>>>>>>>>>>>>>>>> class of
> >>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
> >>>>>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> amount
> >>>>>>>> of
> >>>>>>>>>>>>>>>> methods and it is logical to enable inheritance between
> >>>>>>>>>>>>>>>> those
> >>>>>>>>>>>>>> two.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>> snippet
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> below 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 has gone
> >>>>>>>>>>>>>>>>>> though
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> many
> >>>>>>>> changes
> >>>>>>>>>>>>>>>> based on user/developer needs, both in
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> documentation-/implementation-wise.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
> >>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot
> about
> >>>>>>>>>>>>>>>>>> KAFKA-3907
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> this 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:
> >>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
> >>>>>>>>>>>>>>>>>>> ProcessorContext,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>> enforce
> >>>>>>>>>>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And
> internal
> >>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
> >>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> changed
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> this call.
> >>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
> >>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>> snippet
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> below would need to be updated as well.
> >>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> matthias@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.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> -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
> >>>>>>>>>>>>>>>>>>> got
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>> approval [2,3].
> >>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
> >>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+
> Streams
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> [2]
> >>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
> >>>>>>>>>>>>>>>>>>>>> ka/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. Sax <
> >>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>> 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 DSL users should not
> worry
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert
> a
> >>>>>>>>>>>>>>>>>>>>> dummy
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `transform` to request a commit from there.
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> Rich
> >>>>>>>>>>>>>>>>> methods:
> >>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
> >>>>>>>>>>>>>>>>>>>>>>>                  final K key,
> >>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext
> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> we can adopt
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
> >>>>>>>>>>>>>>>>>>>>>>>                  final K key,
> >>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext recordContext,
> >>>>>>>>>>>>>>>>>>>>>>>                  final ProcessorContext
> >>>>>>>>>>>>>>>>>>>>>>> processorContext);
> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
> >>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/
> jira/browse/KAFKA-3907
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>> itself.
> >>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>>>>>         VR apply(final V1 value1, final V2 value2,
> >>>>>>>>>>>>>>>>>>>>>>>>>> final K
> >>>>>>>>>>>>>>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> - Fixed.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
> >>>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>>> pairing
> >>>>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
> >>>>>>>>>>>>>>>>>>>>>>>> super
> >>>>>>>>>>>>>>> K,
> >>>>>>>>>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>>>>>>>>>>>>>>>>                                        final
> >>>>>>>>>>>>>>>>>>>>>>>>>> Serde<KR>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
> >>>>>>>>>>>>>>>>>>>>>>>>                                        final
> >>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
> >>>>>>>>>>>>>>> valSerde);
> >>>>>>>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K,
> >>>>>>>>>>>>>>>>>>>>>>>> VT>
> >>>>>>>>>>>>>> table,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>                                      final
> >>>>>>>>>>>>>>>>>>>>>>>>> RichValueJoiner<?
> >>>>>>>> super
> >>>>>>>>>>>>>>>>>>>>>>>> K,
> >>>>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>>>>>>>>>>>>>>>>                                      final
> >>>>>>>>>>>>>>>>>>>>>>>>>> Serde<K>
> >>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
> >>>>>>>>>>>>>>>>>>>>>>>>>>                                      final
> >>>>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
> >>>>>>>>>>>>>>>>>>>>>>>>>> valSerde);
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> -Fixed
> >>>>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding
> >>>>>>>>>>>>>>>>>>>>>>>>> three APIs
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>
> >
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I agree with Guozhang that just exposing meta data at the source level
might not provide too much value. Furthermore, for timestamps we do
already have a well defined contract and we should exploit it:
timestamps can always be provided in a meaningful way.

Also, for simple operations like KStream-filter/map the contract is
simple and we can just use it. Same for KTable-filter/map (for new values).

For aggregations, join, and oldValue, I could just drop some information
and return `null`/-1, if the result records has no semantically
meaningful meta data.

For example, for aggregations, we could preserve the partition (as all
agg-input-records have the same partition). For single input topic
aggregation (what I guess is the most prominent case), we can also carry
over the topic name (would be a internal repartitioning topic name
often). Offsets don't have any semantic interpretation IMHO and we could
return -1.

For joins, we could keep the partition information. Topic and offset are
both unknown/invalid for the output record IMHO.

For the oldValue case, we can keep partition and for single input topic
case topic name. Timestamp might be -1 for now, but after we added
timestamps to KTable (what we plan to do anyway), we can also return a
valid timestamp. Offset would be -1 again (if we store offset in KTable
too, we could provide all offset as well -- but I don't see too much
value in doing this compared to the storage overhead this implies).


WDYT?


-Matthias

On 11/29/17 4:14 AM, Jan Filipiak wrote:
> Hi,
> 
> thank you for the summary and thanks for acknowledging that I do have a
> point here.
> 
> I don't like the second Idea at all. Hence I started of this discussion.
> 
> I am just disappointed, back then when we had the discussion about how
> to refactor store overload
> and IQ handling, I knew the path we are taking is wrong. Having problems
> implementing these kinda
> features (wich are really simple)  is just a symptom of messed up IQ
> implementation. I wish really bad
> I could have convinced you guys back then. To be honest with IQ we can
> continue here
> as we Materialize but would not send oldValue, but with join you're out
> of luck with current setup.
> 
> I of course recommend to do not introduce any optimizations here. Id
> recommend to go towards what
> I recommended already back then. So i would't say we need to optimize
> anything later we need to build
> the topology better in the first place.
> 
> 
> 
> 
> On 28.11.2017 21:00, Guozhang Wang wrote:
>> Jan,
>>
>> Thanks for your input, I can understand now that the oldValue is also
>> exposed in user customized `filter` function and hence want record
>> context
>> we should expose is a problem. And I think it does brings a good point to
>> consider for KIP-159. The discussions maybe a bit confusing to reader
>> though, and hence I'd like to summarize the status quo and with a
>> proposal:
>>
>> In today's Streams DSL, when a KTable is created either from a source
>> topic, or from an stateful operator, we will materialize the KTable
>> with a
>> backing state store; on the other hand, KTables created from a
>> non-stateful
>> operator like filter, will not be backed by a state store by default
>> unless
>> users indicate so (e.g. using the overloaded function with the queryable
>> name or store supplier).
>>
>> For example:
>>
>> KTable table1 = builder.table("topic");                              // a
>> state store created for table1
>> KTable table2 = table1.filter(..);
>> // no state store created for table2
>> KTable table3 = table1.filter(.., "storeName");                  // a
>> state
>> store created for table3
>> KTable table4 = table1.groupBy(..).aggregate(..);            // a state
>> store created for table4
>>
>> Because of that, the filter() operator above on table1 will always be
>> exposed with oldValue and newValue; Damian's point is that, we may
>> optimize
>> the first case such that table1 will only be materialized if users
>> asked so
>> (e.g. using the overloaded function with a store supplier), and in which
>> case, we do not need to pass newValue / oldValue pairs (I think this is
>> what Jan suggests as well, i.e. do filtering before materializing, so
>> that
>> we can have a smaller backed state store as well). But this optimization
>> does not eliminate the possibilities that we may still need to do
>> filter if
>> users does specify "yes I do want to the source KTable itself to be
>> materialized, please". So the concern about how to expose the record
>> context in such cases still persists.
>>
>>
>> With that, regarding to KIP-159 itself, here are my thoughts:
>>
>> 1) if we restrict the scope of exposing record context only to source
>> KTables / KStreams I felt the KIP itself does not bring much value given
>> its required API change because only the SourceKStream can safely
>> maintain
>> its records context, and for SourceKTable if it is materialized, then
>> even
>> non-stateful operators like Join may still have a concern about exposing
>> the record context.
>>
>> 2) an alternative idea is we provide the semantics on how record context
>> would be inherited across the operators for KTable / KStream and
>> expose it
>> in all operators (similarly in PAPI we would expose a much simpler
>> contract), and make it as a public contract that Streams library will
>> guarantee moving forward even we optimize our topology builder; it may
>> not
>> align perfectly with the linear algebraic semantics but practically
>> applicable for most cases; if users semantics do not fit in the provided
>> contract, then they may need to get this themselves (embed such
>> information
>> in the value payload, for example).
>>
>> If people do not like the second idea, I'd suggest we hold on pursuing
>> the
>> first direction since to me its beneficial scope is too limited
>> compared to
>> its cost.
>>
>>
>> Guozhang
>>
>>
>>
>> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>>> Cleary we show the oldValue to the user. We have to, because we filter
>>> after the store.
>>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>>> ain/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java#L96
>>>
>>>
>>> I cannot help you following this. It is really obvious and I am running
>>> out of tools for explaining.
>>>
>>> Thanks for understanding my point to put filter before. Not only
>>> would it
>>> make the store smaller. It would make this feature reasonably
>>> possible and
>>> the framework easier. Interestingly it would also help to move IQ
>>> into more
>>> reasonable directions. And it might help understand that we do not
>>> need any
>>> intermediate representation of the topology,
>>>
>>> KIP-182 I have no clue what everyone has with their "bytestores" so
>>> broken. But putting another store after doesn't help when the store
>>> before
>>> is the problem.
>>>
>>>
>>>
>>>
>>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>>
>>>>   From a DSL point of view, users only see the new value on a
>>>> KTable#filter anyway. So why should it be an issue that we use
>>>> <newValue,oldValue> pair under the hood?
>>>>
>>>> User sees newValue and gets corresponding RecordContext. I can't see
>>>> any
>>>> issue here?
>>>>
>>>> I cannot follow here:
>>>>
>>>> Even when we have a statefull operation last. We move it to the very
>>>>>> first processor (KtableSource)
>>>>>> and therefore cant present a proper RecordContext.
>>>>>>
>>>> With regard to `builder.table().filter()`:
>>>>
>>>> I see you point that it would be good to be able to apply the filter()
>>>> first to reduce the stat store size of the table. But how is this
>>>> related to KIP-159?
>>>>
>>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>>> putting a custom dummy store into the table and materialize the filter
>>>> result afterwards? It's not a nice way to do, but seems to be possible.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>>
>>>>> The comment is valid. It falls exactly into this topic, it has exactly
>>>>> todo with this!
>>>>> Even when we have a statefull operation last. We move it to the very
>>>>> first processor (KtableSource)
>>>>> and therefore cant present a proper RecordContext.
>>>>>
>>>>> Regarding the other Jiras you are referring to. They harm the project
>>>>> more than they do good!
>>>>> There is no need for this kind of optimizer and meta representation
>>>>> and
>>>>> what not. I hope they
>>>>> never get implemented.
>>>>>
>>>>> Best Jan
>>>>>
>>>>>
>>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>>
>>>>>> Jan, i think you comment with respect to filtering is valid, though
>>>>>> not for
>>>>>> this KIP. We have separate JIRAs for topology optimization of
>>>>>> which this
>>>>>> falls into.
>>>>>>
>>>>>> Thanks,
>>>>>> Damian
>>>>>>
>>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>> Jan,
>>>>>>> Not sure I understand your argument that "we still going to present
>>>>>>> change.oldValue to the filter even though the record context() is
>>>>>>> for
>>>>>>> change.newValue". Are you referring to `KTableFilter#process()`?
>>>>>>> If yes
>>>>>>> could you point to me which LOC are you concerning about?
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> a remark of mine that got missed during migration:
>>>>>>>> There is this problem that even though we have
>>>>>>>> source.table.filter.join
>>>>>>>> the state-fullness happens at the table step not a the join
>>>>>>>> step. In a
>>>>>>>> filter
>>>>>>>> we still going to present change.oldValue to the filter even though
>>>>>>>> the
>>>>>>>> record context() is for change.newValue. I would go as far as
>>>>>>>> applying
>>>>>>>> the filter before the table processor. Not to just get KIP-159, but
>>>>>>>>
>>>>>>> because
>>>>>>>
>>>>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>>>>> filter
>>>>>>>> 99% of my
>>>>>>>> records. my state could be way smaller. Also widely escalates the
>>>>>>>> context
>>>>>>>> of the KIP
>>>>>>>>
>>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>>
>>>>>>>> Best Jan
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>>
>>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>>>>>> sent
>>>>>>>>> to VOTE thread.
>>>>>>>>>
>>>>>>>>> Copying Guozhang's last summary below. Thanks for this summary.
>>>>>>>>> Very
>>>>>>>>> comprehensive!
>>>>>>>>>
>>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>>> context
>>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>>
>>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>>>>> carrying the context to more operators than just sources. It also
>>>>>>>>> seems,
>>>>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>>>>
>>>>>>>>> I still think, that just going with plain (2) is too
>>>>>>>>> restrictive --
>>>>>>>>> but
>>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>>
>>>>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>>>>> modify
>>>>>>>>> the record and thus for both, it would be absolutely valid to keep
>>>>>>>>> the
>>>>>>>>> context.
>>>>>>>>>
>>>>>>>>> I personally would keep the context for at least all one-to-one
>>>>>>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>>>>>>> context further: at least the offset information is
>>>>>>>>> questionable for
>>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>>> inherited
>>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>>> "partition". Thus, I think it's still valuable information we can
>>>>>>>>> carry
>>>>>>>>> downstreams.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> Jan: which approach are you referring to as "the approach that is
>>>>>>>>> on the
>>>>>>>>>
>>>>>>>>>> table would be perfect"?
>>>>>>>>>>
>>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>>> exposing the
>>>>>>>>>> record context which has the issues that we have been discussing
>>>>>>>>>> right
>>>>>>>>>> now,
>>>>>>>>>> and its semantics is always referring to the "processing
>>>>>>>>>> record" at
>>>>>>>>>>
>>>>>>>>> hand.
>>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>>> different:
>>>>>>>>>>
>>>>>>>>>> 1) the record traversed the topology from source to sink, it
>>>>>>>>>> may be
>>>>>>>>>> transformed into new object or even generate multiple new objects
>>>>>>>>>>
>>>>>>>>> (think:
>>>>>>>> branch) along the traversal. And the record context is referring to
>>>>>>>>> this
>>>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>>>>>>>> entire
>>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>>> treated as
>>>>>>>>>> different transformed values of this record (this applies to join
>>>>>>>>>> and
>>>>>>>>>> aggregations as well).
>>>>>>>>>>
>>>>>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>>>>>> after
>>>>>>>>>> the
>>>>>>>>>> source, and NEW records are forwarded to downstream operators.
>>>>>>>>>> I.e.
>>>>>>>>>>
>>>>>>>>> each
>>>>>>>> record only lives between two adjacent operators, once it
>>>>>>>> reached the
>>>>>>>>> new
>>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>>> I think in the past we have talked about Streams under both
>>>>>>>>>> context,
>>>>>>>>>>
>>>>>>>>> and
>>>>>>>> we
>>>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>>
>>>>>>>>> implementation
>>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>>> ends at
>>>>>>>>>> the
>>>>>>>>>> join operator as it is only be materialized, while stream
>>>>>>>>>> record's
>>>>>>>>>> traversal goes through the join operator to further down until
>>>>>>>>>> sinks).
>>>>>>>>>> However if we are going to interpret following 2) above then even
>>>>>>>>>> for
>>>>>>>>>> non-stateful operators we would not inherit record context. What
>>>>>>>>>> we're
>>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>>
>>>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>>>>>>>>
>>>>>>>>> operators,
>>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>>>>> "mapValues"
>>>>>>>>>>      ) and will "end" at many-to-one (stateful) operators
>>>>>>>>>> where NEW
>>>>>>>>>>
>>>>>>>>> records
>>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>>> Just wanted to lay the ground for discussions so we are all on
>>>>>>>>>> the
>>>>>>>>>> same
>>>>>>>>>> page before chatting more.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>>> Hi Matthias,
>>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past
>>>>>>>>>> designs
>>>>>>>>>>
>>>>>>>>> when
>>>>>>>> punctuate() was not deprecated.
>>>>>>>>>> I corrected.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> I just re-read the KIP.
>>>>>>>>>>
>>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>>> methods.
>>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>>>>> instead
>>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>>
>>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>>
>>>>>>>>>>> Jeyhun,
>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We
>>>>>>>>>>>> should
>>>>>>>>>>>>
>>>>>>>>>>> think
>>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>>>>> moving
>>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>>> conclusion is
>>>>>>>>>>>>
>>>>>>>>>>>> that
>>>>>>>>>>> it would require revamping the public APIs quite a lot, and it's
>>>>>>>>>>>> not
>>>>>>>>>>>>
>>>>>>>>>>>> clear
>>>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>>>>>>>>> instead.
>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy
>>>>>>>>>>>> <da...@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>>
>>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        - on-demand commit() feature
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>> Hi,
>>>>>>>>>>>>
>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It
>>>>>>>>>>>>> simplified
>>>>>>>>>>>>> the
>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't
>>>>>>>>>>>>>> agree that
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and
>>>>>>>>>>>>>>> close the
>>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the best
>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>>>> do.
>>>>>>>>>>>>>> A
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext
>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> an
>>>>>>>> argument
>>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>> `commit()`
>>>>>>>> at
>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you
>>>>>>>>>>>>>>> elaborate a
>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation
>>>>>>>>>>>>>>> better. Why
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> would
>>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> explicitly,
>>>>>>>> ie,
>>>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Requesting
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> a commit is something quite different. Additionally, a
>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>>> specific
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we
>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> this
>>>>>>>> path.
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to
>>>>>>>> enforce
>>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>> intuition that we introduce `commit()` in
>>>>>>>>>>>>>>>> `RecordContext` is
>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>> interfaces.
>>>>>>>> So
>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code
>>>>>>>>>>>>>>>>>> snippet in
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>         public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>>>>>             recordContext = new RecordContext()
>>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>>                 public void commit() {
>>>>>>>>>>>>>>>>>>                     context().commit();
>>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>>                 public long offset() {
>>>>>>>>>>>>>>>>>>                     return context().recordContext().offs
>>>>>>>>>>>>>>>>>> et();
>>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>>                 public long timestamp() {
>>>>>>>>>>>>>>>>>>                     return
>>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>>                 public String topic() {
>>>>>>>>>>>>>>>>>>                     return context().recordContext().topi
>>>>>>>>>>>>>>>>>> c();
>>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>>                 public int partition() {
>>>>>>>>>>>>>>>>>>                     return
>>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>>           };
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> case
>>>>>>>> IMO.
>>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share
>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> amount
>>>>>>>> of
>>>>>>>>>>>>>>>> methods and it is logical to enable inheritance between
>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>> two.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> below 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 has gone
>>>>>>>>>>>>>>>>>> though
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> many
>>>>>>>> changes
>>>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> to
>>>>>>>> enforce
>>>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>>> ka/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. Sax <
>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 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 DSL users should not worry
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -- 
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>>                  final K key,
>>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext recordContext);
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>>                  final K key,
>>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>>>>>                  final ProcessorContext
>>>>>>>>>>>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>         VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>> pairing
>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>>>>>                                        final
>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>                                        final
>>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K,
>>>>>>>>>>>>>>>>>>>>>>>> VT>
>>>>>>>>>>>>>> table,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>                                      final
>>>>>>>>>>>>>>>>>>>>>>>>> RichValueJoiner<?
>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>>>>>                                      final
>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<K>
>>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>>>                                      final
>>>>>>>>>>>>>>>>>>>>>>>>>> Serde<V>
>>>>>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding
>>>>>>>>>>>>>>>>>>>>>>>>> three APIs
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>
>>
> 
> 


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

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi,

thank you for the summary and thanks for acknowledging that I do have a 
point here.

I don't like the second Idea at all. Hence I started of this discussion.

I am just disappointed, back then when we had the discussion about how 
to refactor store overload
and IQ handling, I knew the path we are taking is wrong. Having problems 
implementing these kinda
features (wich are really simple)  is just a symptom of messed up IQ 
implementation. I wish really bad
I could have convinced you guys back then. To be honest with IQ we can 
continue here
as we Materialize but would not send oldValue, but with join you're out 
of luck with current setup.

I of course recommend to do not introduce any optimizations here. Id 
recommend to go towards what
I recommended already back then. So i would't say we need to optimize 
anything later we need to build
the topology better in the first place.




On 28.11.2017 21:00, Guozhang Wang wrote:
> Jan,
>
> Thanks for your input, I can understand now that the oldValue is also
> exposed in user customized `filter` function and hence want record context
> we should expose is a problem. And I think it does brings a good point to
> consider for KIP-159. The discussions maybe a bit confusing to reader
> though, and hence I'd like to summarize the status quo and with a proposal:
>
> In today's Streams DSL, when a KTable is created either from a source
> topic, or from an stateful operator, we will materialize the KTable with a
> backing state store; on the other hand, KTables created from a non-stateful
> operator like filter, will not be backed by a state store by default unless
> users indicate so (e.g. using the overloaded function with the queryable
> name or store supplier).
>
> For example:
>
> KTable table1 = builder.table("topic");                              // a
> state store created for table1
> KTable table2 = table1.filter(..);
> // no state store created for table2
> KTable table3 = table1.filter(.., "storeName");                  // a state
> store created for table3
> KTable table4 = table1.groupBy(..).aggregate(..);            // a state
> store created for table4
>
> Because of that, the filter() operator above on table1 will always be
> exposed with oldValue and newValue; Damian's point is that, we may optimize
> the first case such that table1 will only be materialized if users asked so
> (e.g. using the overloaded function with a store supplier), and in which
> case, we do not need to pass newValue / oldValue pairs (I think this is
> what Jan suggests as well, i.e. do filtering before materializing, so that
> we can have a smaller backed state store as well). But this optimization
> does not eliminate the possibilities that we may still need to do filter if
> users does specify "yes I do want to the source KTable itself to be
> materialized, please". So the concern about how to expose the record
> context in such cases still persists.
>
>
> With that, regarding to KIP-159 itself, here are my thoughts:
>
> 1) if we restrict the scope of exposing record context only to source
> KTables / KStreams I felt the KIP itself does not bring much value given
> its required API change because only the SourceKStream can safely maintain
> its records context, and for SourceKTable if it is materialized, then even
> non-stateful operators like Join may still have a concern about exposing
> the record context.
>
> 2) an alternative idea is we provide the semantics on how record context
> would be inherited across the operators for KTable / KStream and expose it
> in all operators (similarly in PAPI we would expose a much simpler
> contract), and make it as a public contract that Streams library will
> guarantee moving forward even we optimize our topology builder; it may not
> align perfectly with the linear algebraic semantics but practically
> applicable for most cases; if users semantics do not fit in the provided
> contract, then they may need to get this themselves (embed such information
> in the value payload, for example).
>
> If people do not like the second idea, I'd suggest we hold on pursuing the
> first direction since to me its beneficial scope is too limited compared to
> its cost.
>
>
> Guozhang
>
>
>
> On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Cleary we show the oldValue to the user. We have to, because we filter
>> after the store.
>> https://github.com/axbaretto/kafka/blob/master/streams/src/m
>> ain/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java#L96
>>
>> I cannot help you following this. It is really obvious and I am running
>> out of tools for explaining.
>>
>> Thanks for understanding my point to put filter before. Not only would it
>> make the store smaller. It would make this feature reasonably possible and
>> the framework easier. Interestingly it would also help to move IQ into more
>> reasonable directions. And it might help understand that we do not need any
>> intermediate representation of the topology,
>>
>> KIP-182 I have no clue what everyone has with their "bytestores" so
>> broken. But putting another store after doesn't help when the store before
>> is the problem.
>>
>>
>>
>>
>> On 24.11.2017 05:08, Matthias J. Sax wrote:
>>
>>>   From a DSL point of view, users only see the new value on a
>>> KTable#filter anyway. So why should it be an issue that we use
>>> <newValue,oldValue> pair under the hood?
>>>
>>> User sees newValue and gets corresponding RecordContext. I can't see any
>>> issue here?
>>>
>>> I cannot follow here:
>>>
>>> Even when we have a statefull operation last. We move it to the very
>>>>> first processor (KtableSource)
>>>>> and therefore cant present a proper RecordContext.
>>>>>
>>> With regard to `builder.table().filter()`:
>>>
>>> I see you point that it would be good to be able to apply the filter()
>>> first to reduce the stat store size of the table. But how is this
>>> related to KIP-159?
>>>
>>> Btw: with KIP-182, I am wondering if this would not be possible, by
>>> putting a custom dummy store into the table and materialize the filter
>>> result afterwards? It's not a nice way to do, but seems to be possible.
>>>
>>>
>>> -Matthias
>>>
>>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>>
>>>> The comment is valid. It falls exactly into this topic, it has exactly
>>>> todo with this!
>>>> Even when we have a statefull operation last. We move it to the very
>>>> first processor (KtableSource)
>>>> and therefore cant present a proper RecordContext.
>>>>
>>>> Regarding the other Jiras you are referring to. They harm the project
>>>> more than they do good!
>>>> There is no need for this kind of optimizer and meta representation and
>>>> what not. I hope they
>>>> never get implemented.
>>>>
>>>> Best Jan
>>>>
>>>>
>>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>>
>>>>> Jan, i think you comment with respect to filtering is valid, though
>>>>> not for
>>>>> this KIP. We have separate JIRAs for topology optimization of which this
>>>>> falls into.
>>>>>
>>>>> Thanks,
>>>>> Damian
>>>>>
>>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:
>>>>>
>>>>> Jan,
>>>>>> Not sure I understand your argument that "we still going to present
>>>>>> change.oldValue to the filter even though the record context() is for
>>>>>> change.newValue". Are you referring to `KTableFilter#process()`? If yes
>>>>>> could you point to me which LOC are you concerning about?
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>>> Jan.Filipiak@trivago.com>
>>>>>> wrote:
>>>>>>
>>>>>> a remark of mine that got missed during migration:
>>>>>>> There is this problem that even though we have
>>>>>>> source.table.filter.join
>>>>>>> the state-fullness happens at the table step not a the join step. In a
>>>>>>> filter
>>>>>>> we still going to present change.oldValue to the filter even though
>>>>>>> the
>>>>>>> record context() is for change.newValue. I would go as far as applying
>>>>>>> the filter before the table processor. Not to just get KIP-159, but
>>>>>>>
>>>>>> because
>>>>>>
>>>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>>>> filter
>>>>>>> 99% of my
>>>>>>> records. my state could be way smaller. Also widely escalates the
>>>>>>> context
>>>>>>> of the KIP
>>>>>>>
>>>>>>> I can only see upsides of executing the filter first.
>>>>>>>
>>>>>>> Best Jan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>>
>>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>>>>> sent
>>>>>>>> to VOTE thread.
>>>>>>>>
>>>>>>>> Copying Guozhang's last summary below. Thanks for this summary. Very
>>>>>>>> comprehensive!
>>>>>>>>
>>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>>> context
>>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>>
>>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>>>> carrying the context to more operators than just sources. It also
>>>>>>>> seems,
>>>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>>>
>>>>>>>> I still think, that just going with plain (2) is too restrictive --
>>>>>>>> but
>>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>>
>>>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>>>> modify
>>>>>>>> the record and thus for both, it would be absolutely valid to keep
>>>>>>>> the
>>>>>>>> context.
>>>>>>>>
>>>>>>>> I personally would keep the context for at least all one-to-one
>>>>>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>>>>>> context further: at least the offset information is questionable for
>>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>>> inherited
>>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>>> "partition". Thus, I think it's still valuable information we can
>>>>>>>> carry
>>>>>>>> downstreams.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> Jan: which approach are you referring to as "the approach that is
>>>>>>>> on the
>>>>>>>>
>>>>>>>>> table would be perfect"?
>>>>>>>>>
>>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>>> exposing the
>>>>>>>>> record context which has the issues that we have been discussing
>>>>>>>>> right
>>>>>>>>> now,
>>>>>>>>> and its semantics is always referring to the "processing record" at
>>>>>>>>>
>>>>>>>> hand.
>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>>> different:
>>>>>>>>>
>>>>>>>>> 1) the record traversed the topology from source to sink, it may be
>>>>>>>>> transformed into new object or even generate multiple new objects
>>>>>>>>>
>>>>>>>> (think:
>>>>>>> branch) along the traversal. And the record context is referring to
>>>>>>>> this
>>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>>>>>>> entire
>>>>>>> topology traversal and any new records of this traversal is
>>>>>>>>> treated as
>>>>>>>>> different transformed values of this record (this applies to join
>>>>>>>>> and
>>>>>>>>> aggregations as well).
>>>>>>>>>
>>>>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>>>>> after
>>>>>>>>> the
>>>>>>>>> source, and NEW records are forwarded to downstream operators. I.e.
>>>>>>>>>
>>>>>>>> each
>>>>>>> record only lives between two adjacent operators, once it reached the
>>>>>>>> new
>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>> I think in the past we have talked about Streams under both context,
>>>>>>>>>
>>>>>>>> and
>>>>>>> we
>>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>>
>>>>>>>> implementation
>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>>> ends at
>>>>>>>>> the
>>>>>>>>> join operator as it is only be materialized, while stream record's
>>>>>>>>> traversal goes through the join operator to further down until
>>>>>>>>> sinks).
>>>>>>>>> However if we are going to interpret following 2) above then even
>>>>>>>>> for
>>>>>>>>> non-stateful operators we would not inherit record context. What
>>>>>>>>> we're
>>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>>
>>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>>>>>>>
>>>>>>>> operators,
>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>>>> "mapValues"
>>>>>>>>>      ) and will "end" at many-to-one (stateful) operators where NEW
>>>>>>>>>
>>>>>>>> records
>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>> Just wanted to lay the ground for discussions so we are all on the
>>>>>>>>> same
>>>>>>>>> page before chatting more.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>> Hi Matthias,
>>>>>>>>> Thanks a lot for correcting. It is a leftover from the past designs
>>>>>>>>>
>>>>>>>> when
>>>>>>> punctuate() was not deprecated.
>>>>>>>>> I corrected.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Jeyhun
>>>>>>>>>
>>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>>> <ma...@confluent.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> I just re-read the KIP.
>>>>>>>>>
>>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>>> methods.
>>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>>>> instead
>>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>>
>>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>>
>>>>>>>>>> Thanks for being so patient!
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>>
>>>>>>>>>> Jeyhun,
>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
>>>>>>>>>>>
>>>>>>>>>> think
>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>>>> moving
>>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>>> conclusion is
>>>>>>>>>>>
>>>>>>>>>>> that
>>>>>>>>>> it would require revamping the public APIs quite a lot, and it's
>>>>>>>>>>> not
>>>>>>>>>>>
>>>>>>>>>>> clear
>>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>>>>>>>> instead.
>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>>
>>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>>
>>>>>>>>>>>>        - on-demand commit() feature
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Damian
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
>>>>>>>>>>>> the
>>>>>>> overall design of KIP a lot.
>>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>>>>>>
>>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the best
>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> to
>>>>>>> do.
>>>>>>>>>>>>> A
>>>>>>>>>>>>>
>>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> an
>>>>>>> argument
>>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>> to
>>>>>>>>>>>>>
>>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>> `commit()`
>>>>>>> at
>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>> in the DSL? Just want to understand the motivation better. Why
>>>>>>>>>>>>>>
>>>>>>>>>>>>> would
>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> explicitly,
>>>>>>> ie,
>>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Requesting
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> a commit is something quite different. Additionally, a commit
>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>>> specific
>>>>>>>>>>>>>
>>>>>>>>>>>>>> record.
>>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we follow
>>>>>>>>>>>>>>
>>>>>>>>>>>>> this
>>>>>>> path.
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> to
>>>>>>> enforce
>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is
>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>> interfaces.
>>>>>>> So
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>>>         public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>>>>             recordContext = new RecordContext()
>>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>                 public void commit() {
>>>>>>>>>>>>>>>>>                     context().commit();
>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>                 public long offset() {
>>>>>>>>>>>>>>>>>                     return context().recordContext().offs
>>>>>>>>>>>>>>>>> et();
>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>                 public long timestamp() {
>>>>>>>>>>>>>>>>>                     return
>>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>                 public String topic() {
>>>>>>>>>>>>>>>>>                     return context().recordContext().topi
>>>>>>>>>>>>>>>>> c();
>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>                 @Override
>>>>>>>>>>>>>>>>>                 public int partition() {
>>>>>>>>>>>>>>>>>                     return
>>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>>                 }
>>>>>>>>>>>>>>>>>           };
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> case
>>>>>>> IMO.
>>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> amount
>>>>>>> of
>>>>>>>>>>>>>>> methods and it is logical to enable inheritance between those
>>>>>>>>>>>>> two.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> below 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 has gone though
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> many
>>>>>>> changes
>>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to
>>>>>>> enforce
>>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>>> ka/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. Sax <
>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 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 DSL users should not worry
>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>                  final K key,
>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext recordContext);
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>         VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>>                  final K key,
>>>>>>>>>>>>>>>>>>>>>>                  final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>>>>                  final ProcessorContext
>>>>>>>>>>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>> single
>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>         VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>> pairing
>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>>>>                                        final
>>>>>>>>>>>>>>>>>>>>>>>>> Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>                                        final Serde<V>
>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT>
>>>>>>>>>>>>> table,
>>>>>>>>>>>>>
>>>>>>>>>>>>>>                                      final
>>>>>>>>>>>>>>>>>>>>>>>> RichValueJoiner<?
>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>>>>                                      final Serde<K>
>>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>>                                      final Serde<V>
>>>>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>
>



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

Posted by Guozhang Wang <wa...@gmail.com>.
Jan,

Thanks for your input, I can understand now that the oldValue is also
exposed in user customized `filter` function and hence want record context
we should expose is a problem. And I think it does brings a good point to
consider for KIP-159. The discussions maybe a bit confusing to reader
though, and hence I'd like to summarize the status quo and with a proposal:

In today's Streams DSL, when a KTable is created either from a source
topic, or from an stateful operator, we will materialize the KTable with a
backing state store; on the other hand, KTables created from a non-stateful
operator like filter, will not be backed by a state store by default unless
users indicate so (e.g. using the overloaded function with the queryable
name or store supplier).

For example:

KTable table1 = builder.table("topic");                              // a
state store created for table1
KTable table2 = table1.filter(..);
// no state store created for table2
KTable table3 = table1.filter(.., "storeName");                  // a state
store created for table3
KTable table4 = table1.groupBy(..).aggregate(..);            // a state
store created for table4

Because of that, the filter() operator above on table1 will always be
exposed with oldValue and newValue; Damian's point is that, we may optimize
the first case such that table1 will only be materialized if users asked so
(e.g. using the overloaded function with a store supplier), and in which
case, we do not need to pass newValue / oldValue pairs (I think this is
what Jan suggests as well, i.e. do filtering before materializing, so that
we can have a smaller backed state store as well). But this optimization
does not eliminate the possibilities that we may still need to do filter if
users does specify "yes I do want to the source KTable itself to be
materialized, please". So the concern about how to expose the record
context in such cases still persists.


With that, regarding to KIP-159 itself, here are my thoughts:

1) if we restrict the scope of exposing record context only to source
KTables / KStreams I felt the KIP itself does not bring much value given
its required API change because only the SourceKStream can safely maintain
its records context, and for SourceKTable if it is materialized, then even
non-stateful operators like Join may still have a concern about exposing
the record context.

2) an alternative idea is we provide the semantics on how record context
would be inherited across the operators for KTable / KStream and expose it
in all operators (similarly in PAPI we would expose a much simpler
contract), and make it as a public contract that Streams library will
guarantee moving forward even we optimize our topology builder; it may not
align perfectly with the linear algebraic semantics but practically
applicable for most cases; if users semantics do not fit in the provided
contract, then they may need to get this themselves (embed such information
in the value payload, for example).

If people do not like the second idea, I'd suggest we hold on pursuing the
first direction since to me its beneficial scope is too limited compared to
its cost.


Guozhang



On Fri, Nov 24, 2017 at 1:39 AM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Cleary we show the oldValue to the user. We have to, because we filter
> after the store.
> https://github.com/axbaretto/kafka/blob/master/streams/src/m
> ain/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java#L96
>
> I cannot help you following this. It is really obvious and I am running
> out of tools for explaining.
>
> Thanks for understanding my point to put filter before. Not only would it
> make the store smaller. It would make this feature reasonably possible and
> the framework easier. Interestingly it would also help to move IQ into more
> reasonable directions. And it might help understand that we do not need any
> intermediate representation of the topology,
>
> KIP-182 I have no clue what everyone has with their "bytestores" so
> broken. But putting another store after doesn't help when the store before
> is the problem.
>
>
>
>
> On 24.11.2017 05:08, Matthias J. Sax wrote:
>
>>  From a DSL point of view, users only see the new value on a
>> KTable#filter anyway. So why should it be an issue that we use
>> <newValue,oldValue> pair under the hood?
>>
>> User sees newValue and gets corresponding RecordContext. I can't see any
>> issue here?
>>
>> I cannot follow here:
>>
>> Even when we have a statefull operation last. We move it to the very
>>>> first processor (KtableSource)
>>>> and therefore cant present a proper RecordContext.
>>>>
>>>
>>
>> With regard to `builder.table().filter()`:
>>
>> I see you point that it would be good to be able to apply the filter()
>> first to reduce the stat store size of the table. But how is this
>> related to KIP-159?
>>
>> Btw: with KIP-182, I am wondering if this would not be possible, by
>> putting a custom dummy store into the table and materialize the filter
>> result afterwards? It's not a nice way to do, but seems to be possible.
>>
>>
>> -Matthias
>>
>> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>>
>>> The comment is valid. It falls exactly into this topic, it has exactly
>>> todo with this!
>>> Even when we have a statefull operation last. We move it to the very
>>> first processor (KtableSource)
>>> and therefore cant present a proper RecordContext.
>>>
>>> Regarding the other Jiras you are referring to. They harm the project
>>> more than they do good!
>>> There is no need for this kind of optimizer and meta representation and
>>> what not. I hope they
>>> never get implemented.
>>>
>>> Best Jan
>>>
>>>
>>> On 22.11.2017 14:44, Damian Guy wrote:
>>>
>>>> Jan, i think you comment with respect to filtering is valid, though
>>>> not for
>>>> this KIP. We have separate JIRAs for topology optimization of which this
>>>> falls into.
>>>>
>>>> Thanks,
>>>> Damian
>>>>
>>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:
>>>>
>>>> Jan,
>>>>>
>>>>> Not sure I understand your argument that "we still going to present
>>>>> change.oldValue to the filter even though the record context() is for
>>>>> change.newValue". Are you referring to `KTableFilter#process()`? If yes
>>>>> could you point to me which LOC are you concerning about?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <
>>>>> Jan.Filipiak@trivago.com>
>>>>> wrote:
>>>>>
>>>>> a remark of mine that got missed during migration:
>>>>>>
>>>>>> There is this problem that even though we have
>>>>>> source.table.filter.join
>>>>>> the state-fullness happens at the table step not a the join step. In a
>>>>>> filter
>>>>>> we still going to present change.oldValue to the filter even though
>>>>>> the
>>>>>> record context() is for change.newValue. I would go as far as applying
>>>>>> the filter before the table processor. Not to just get KIP-159, but
>>>>>>
>>>>> because
>>>>>
>>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>>> filter
>>>>>> 99% of my
>>>>>> records. my state could be way smaller. Also widely escalates the
>>>>>> context
>>>>>> of the KIP
>>>>>>
>>>>>> I can only see upsides of executing the filter first.
>>>>>>
>>>>>> Best Jan
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>>
>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>>>> sent
>>>>>>> to VOTE thread.
>>>>>>>
>>>>>>> Copying Guozhang's last summary below. Thanks for this summary. Very
>>>>>>> comprehensive!
>>>>>>>
>>>>>>> It seems, we all agree, that the current implementation of the
>>>>>>> context
>>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>>
>>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>>> carrying the context to more operators than just sources. It also
>>>>>>> seems,
>>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>>
>>>>>>> I still think, that just going with plain (2) is too restrictive --
>>>>>>> but
>>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>>
>>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>>> modify
>>>>>>> the record and thus for both, it would be absolutely valid to keep
>>>>>>> the
>>>>>>> context.
>>>>>>>
>>>>>>> I personally would keep the context for at least all one-to-one
>>>>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>>>>> context further: at least the offset information is questionable for
>>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>>> inherited
>>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>>> "partition". Thus, I think it's still valuable information we can
>>>>>>> carry
>>>>>>> downstreams.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> Jan: which approach are you referring to as "the approach that is
>>>>>>> on the
>>>>>>>
>>>>>>>> table would be perfect"?
>>>>>>>>
>>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>>> exposing the
>>>>>>>> record context which has the issues that we have been discussing
>>>>>>>> right
>>>>>>>> now,
>>>>>>>> and its semantics is always referring to the "processing record" at
>>>>>>>>
>>>>>>> hand.
>>>>>
>>>>>> More specifically, we can think of processing a record a bit
>>>>>>>> different:
>>>>>>>>
>>>>>>>> 1) the record traversed the topology from source to sink, it may be
>>>>>>>> transformed into new object or even generate multiple new objects
>>>>>>>>
>>>>>>> (think:
>>>>>
>>>>>> branch) along the traversal. And the record context is referring to
>>>>>>>>
>>>>>>> this
>>>>>
>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>>>>>>>
>>>>>>> entire
>>>>>
>>>>>> topology traversal and any new records of this traversal is
>>>>>>>> treated as
>>>>>>>> different transformed values of this record (this applies to join
>>>>>>>> and
>>>>>>>> aggregations as well).
>>>>>>>>
>>>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>>>> after
>>>>>>>> the
>>>>>>>> source, and NEW records are forwarded to downstream operators. I.e.
>>>>>>>>
>>>>>>> each
>>>>>
>>>>>> record only lives between two adjacent operators, once it reached the
>>>>>>>>
>>>>>>> new
>>>>>
>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>>
>>>>>>>> I think in the past we have talked about Streams under both context,
>>>>>>>>
>>>>>>> and
>>>>>
>>>>>> we
>>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>>> understandable for users as it does not leak any internal
>>>>>>>>
>>>>>>> implementation
>>>>>
>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>>> ends at
>>>>>>>> the
>>>>>>>> join operator as it is only be materialized, while stream record's
>>>>>>>> traversal goes through the join operator to further down until
>>>>>>>> sinks).
>>>>>>>> However if we are going to interpret following 2) above then even
>>>>>>>> for
>>>>>>>> non-stateful operators we would not inherit record context. What
>>>>>>>> we're
>>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>>
>>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>>>>>>
>>>>>>> operators,
>>>>>
>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>>> "mapValues"
>>>>>>>>     ) and will "end" at many-to-one (stateful) operators where NEW
>>>>>>>>
>>>>>>> records
>>>>>
>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>>
>>>>>>>> Just wanted to lay the ground for discussions so we are all on the
>>>>>>>> same
>>>>>>>> page before chatting more.
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>>
>>>>>>> Hi Matthias,
>>>>>>>>
>>>>>>>> Thanks a lot for correcting. It is a leftover from the past designs
>>>>>>>>
>>>>>>> when
>>>>>
>>>>>> punctuate() was not deprecated.
>>>>>>>> I corrected.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>>> <ma...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> I just re-read the KIP.
>>>>>>>>
>>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>>> methods.
>>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>>> instead
>>>>>>>>> of introducing it as deprecated.
>>>>>>>>>
>>>>>>>>> Otherwise looks good to me.
>>>>>>>>>
>>>>>>>>> Thanks for being so patient!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>>
>>>>>>>>> Jeyhun,
>>>>>>>>>>
>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
>>>>>>>>>>
>>>>>>>>> think
>>>>>
>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>>> moving
>>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>>> conclusion is
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>
>>>>>>>>> it would require revamping the public APIs quite a lot, and it's
>>>>>>>>>> not
>>>>>>>>>>
>>>>>>>>>> clear
>>>>>>>>>
>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>>>>>>>>
>>>>>>>>> instead.
>>>>>
>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>>
>>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>>
>>>>>>>>>>>       - on-demand commit() feature
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <
>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>> Hi,
>>>>>>>>>>
>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
>>>>>>>>>>>>
>>>>>>>>>>> the
>>>>>
>>>>>> overall design of KIP a lot.
>>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>>>>>
>>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the best
>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>
>>>>>>>>>>>>> to
>>>>>
>>>>>> do.
>>>>>>>>>>>>>
>>>>>>>>>>>> A
>>>>>>>>>>>>
>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself
>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>
>>>>>>>>>>>>> an
>>>>>
>>>>>> argument
>>>>>>>>>>>>>
>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor
>>>>>>>>>>>>> level
>>>>>>>>>>>>>
>>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>>> need ij
>>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> and
>>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>>> provide
>>>>>>>>>>>>>
>>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>>
>>>>>>>>>>>>> ).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>>
>>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> KIP/included
>>>>>>>>>>>>>
>>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>>>>>>>>>>>>
>>>>>>>>>>>>> `commit()`
>>>>>
>>>>>> at
>>>>>>>>>>>>>>
>>>>>>>>>>>>> all.
>>>>>>>>>>>>
>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> exposed
>>>>>>>>>>>>>>
>>>>>>>>>>>>> in the DSL? Just want to understand the motivation better. Why
>>>>>>>>>>>>>
>>>>>>>>>>>> would
>>>>>
>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> explicitly,
>>>>>
>>>>>> ie,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>
>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>
>>>>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Requesting
>>>>>>>>>>>>>>
>>>>>>>>>>>>> a commit is something quite different. Additionally, a commit
>>>>>>>>>>>>> does
>>>>>>>>>>>>>
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>
>>>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a
>>>>>>>>>>>> specific
>>>>>>>>>>>>
>>>>>>>>>>>>> record.
>>>>>>>>>>>>>>
>>>>>>>>>>>>> To me, this does not seem to be a sound API design if we follow
>>>>>>>>>>>>>
>>>>>>>>>>>> this
>>>>>
>>>>>> path.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>
>>>>>> enforce
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>
>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is
>>>>>>>>>>>>> that,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> interfaces.
>>>>>
>>>>>> So
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> user
>>>>>>>>>>>>>
>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> @Override
>>>>>>>>>>>>
>>>>>>>>>>>>>        public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>            recordContext = new RecordContext()
>>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>>                public void commit() {
>>>>>>>>>>>>>>>>                    context().commit();
>>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>>                public long offset() {
>>>>>>>>>>>>>>>>                    return context().recordContext().offs
>>>>>>>>>>>>>>>> et();
>>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>>                public long timestamp() {
>>>>>>>>>>>>>>>>                    return
>>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>>                public String topic() {
>>>>>>>>>>>>>>>>                    return context().recordContext().topi
>>>>>>>>>>>>>>>> c();
>>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>>                public int partition() {
>>>>>>>>>>>>>>>>                    return
>>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>>          };
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> case
>>>>>
>>>>>> IMO.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> amount
>>>>>
>>>>>> of
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> methods and it is logical to enable inheritance between those
>>>>>>>>>>>> two.
>>>>>>>>>>>>
>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> below 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 has gone though
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> many
>>>>>
>>>>>> changes
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>
>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>
>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>>> ProcessorContext,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> to
>>>>>
>>>>>> enforce
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> changed
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>
>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>
>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> approval [2,3].
>>>>>>>>>>>>>>>>>>> Maybe I misunderstood.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>> ka/uyzND19Asmg1GKKXT1?subj=
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>> http://search-hadoop.com/m/Kaf
>>>>>>>>>>>>>>>>>>> ka/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. Sax <
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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 DSL users should not worry
>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>
>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>
>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>        VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>                 final K key,
>>>>>>>>>>>>>>>>>>>>>                 final RecordContext recordContext);
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>>        VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>>                 final K key,
>>>>>>>>>>>>>>>>>>>>>                 final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>>>                 final ProcessorContext
>>>>>>>>>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>
>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> that
>>>>>
>>>>>> single
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>
>>>>>>>>>>>>>        VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> pairing
>>>>>>>>>>>>
>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>
>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>                                       final
>>>>>>>>>>>>>>>>>>>>>>>> Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>                                       final Serde<V>
>>>>>>>>>>>>
>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT>
>>>>>>>>>>>> table,
>>>>>>>>>>>>
>>>>>>>>>>>>>                                     final
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> RichValueJoiner<?
>>>>>
>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>>>                                     final Serde<K>
>>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>>                                     final Serde<V>
>>>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>
>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>


-- 
-- Guozhang

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

Posted by Jan Filipiak <Ja...@trivago.com>.
Cleary we show the oldValue to the user. We have to, because we filter 
after the store.
https://github.com/axbaretto/kafka/blob/master/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java#L96

I cannot help you following this. It is really obvious and I am running 
out of tools for explaining.

Thanks for understanding my point to put filter before. Not only would 
it make the store smaller. It would make this feature reasonably 
possible and the framework easier. Interestingly it would also help to 
move IQ into more reasonable directions. And it might help understand 
that we do not need any intermediate representation of the topology,

KIP-182 I have no clue what everyone has with their "bytestores" so 
broken. But putting another store after doesn't help when the store 
before is the problem.




On 24.11.2017 05:08, Matthias J. Sax wrote:
>  From a DSL point of view, users only see the new value on a
> KTable#filter anyway. So why should it be an issue that we use
> <newValue,oldValue> pair under the hood?
>
> User sees newValue and gets corresponding RecordContext. I can't see any
> issue here?
>
> I cannot follow here:
>
>>> Even when we have a statefull operation last. We move it to the very
>>> first processor (KtableSource)
>>> and therefore cant present a proper RecordContext.
>
>
> With regard to `builder.table().filter()`:
>
> I see you point that it would be good to be able to apply the filter()
> first to reduce the stat store size of the table. But how is this
> related to KIP-159?
>
> Btw: with KIP-182, I am wondering if this would not be possible, by
> putting a custom dummy store into the table and materialize the filter
> result afterwards? It's not a nice way to do, but seems to be possible.
>
>
> -Matthias
>
> On 11/23/17 4:56 AM, Jan Filipiak wrote:
>> The comment is valid. It falls exactly into this topic, it has exactly
>> todo with this!
>> Even when we have a statefull operation last. We move it to the very
>> first processor (KtableSource)
>> and therefore cant present a proper RecordContext.
>>
>> Regarding the other Jiras you are referring to. They harm the project
>> more than they do good!
>> There is no need for this kind of optimizer and meta representation and
>> what not. I hope they
>> never get implemented.
>>
>> Best Jan
>>
>>
>> On 22.11.2017 14:44, Damian Guy wrote:
>>> Jan, i think you comment with respect to filtering is valid, though
>>> not for
>>> this KIP. We have separate JIRAs for topology optimization of which this
>>> falls into.
>>>
>>> Thanks,
>>> Damian
>>>
>>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:
>>>
>>>> Jan,
>>>>
>>>> Not sure I understand your argument that "we still going to present
>>>> change.oldValue to the filter even though the record context() is for
>>>> change.newValue". Are you referring to `KTableFilter#process()`? If yes
>>>> could you point to me which LOC are you concerning about?
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>> a remark of mine that got missed during migration:
>>>>>
>>>>> There is this problem that even though we have source.table.filter.join
>>>>> the state-fullness happens at the table step not a the join step. In a
>>>>> filter
>>>>> we still going to present change.oldValue to the filter even though the
>>>>> record context() is for change.newValue. I would go as far as applying
>>>>> the filter before the table processor. Not to just get KIP-159, but
>>>> because
>>>>> I think its a side effect of a non ideal topology layout. If i can
>>>>> filter
>>>>> 99% of my
>>>>> records. my state could be way smaller. Also widely escalates the
>>>>> context
>>>>> of the KIP
>>>>>
>>>>> I can only see upsides of executing the filter first.
>>>>>
>>>>> Best Jan
>>>>>
>>>>>
>>>>>
>>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>>
>>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>>> sent
>>>>>> to VOTE thread.
>>>>>>
>>>>>> Copying Guozhang's last summary below. Thanks for this summary. Very
>>>>>> comprehensive!
>>>>>>
>>>>>> It seems, we all agree, that the current implementation of the context
>>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>>
>>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>>> carrying the context to more operators than just sources. It also
>>>>>> seems,
>>>>>> that we all agree, that many-to-one operations void the context.
>>>>>>
>>>>>> I still think, that just going with plain (2) is too restrictive --
>>>>>> but
>>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>>
>>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>>> modify
>>>>>> the record and thus for both, it would be absolutely valid to keep the
>>>>>> context.
>>>>>>
>>>>>> I personally would keep the context for at least all one-to-one
>>>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>>>> context further: at least the offset information is questionable for
>>>>>> this case -- note thought, that semantically, the timestamp is
>>>>>> inherited
>>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>>> "partition". Thus, I think it's still valuable information we can
>>>>>> carry
>>>>>> downstreams.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> Jan: which approach are you referring to as "the approach that is
>>>>>> on the
>>>>>>> table would be perfect"?
>>>>>>>
>>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>>> exposing the
>>>>>>> record context which has the issues that we have been discussing
>>>>>>> right
>>>>>>> now,
>>>>>>> and its semantics is always referring to the "processing record" at
>>>> hand.
>>>>>>> More specifically, we can think of processing a record a bit
>>>>>>> different:
>>>>>>>
>>>>>>> 1) the record traversed the topology from source to sink, it may be
>>>>>>> transformed into new object or even generate multiple new objects
>>>> (think:
>>>>>>> branch) along the traversal. And the record context is referring to
>>>> this
>>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>>> entire
>>>>>>> topology traversal and any new records of this traversal is
>>>>>>> treated as
>>>>>>> different transformed values of this record (this applies to join and
>>>>>>> aggregations as well).
>>>>>>>
>>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>>> after
>>>>>>> the
>>>>>>> source, and NEW records are forwarded to downstream operators. I.e.
>>>> each
>>>>>>> record only lives between two adjacent operators, once it reached the
>>>> new
>>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>>
>>>>>>> I think in the past we have talked about Streams under both context,
>>>> and
>>>>>>> we
>>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>>> understandable for users as it does not leak any internal
>>>> implementation
>>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>>> ends at
>>>>>>> the
>>>>>>> join operator as it is only be materialized, while stream record's
>>>>>>> traversal goes through the join operator to further down until
>>>>>>> sinks).
>>>>>>> However if we are going to interpret following 2) above then even for
>>>>>>> non-stateful operators we would not inherit record context. What
>>>>>>> we're
>>>>>>> discussing now, seems to infer a third semantics:
>>>>>>>
>>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>>> operators,
>>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>>> "mapValues"
>>>>>>>     ) and will "end" at many-to-one (stateful) operators where NEW
>>>> records
>>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>>
>>>>>>> Just wanted to lay the ground for discussions so we are all on the
>>>>>>> same
>>>>>>> page before chatting more.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>>
>>>>>>> Hi Matthias,
>>>>>>>
>>>>>>> Thanks a lot for correcting. It is a leftover from the past designs
>>>> when
>>>>>>> punctuate() was not deprecated.
>>>>>>> I corrected.
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Jeyhun
>>>>>>>
>>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>>> <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>> I just re-read the KIP.
>>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>>> methods.
>>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>>> instead
>>>>>>>> of introducing it as deprecated.
>>>>>>>>
>>>>>>>> Otherwise looks good to me.
>>>>>>>>
>>>>>>>> Thanks for being so patient!
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>>
>>>>>>>>> Jeyhun,
>>>>>>>>>
>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
>>>> think
>>>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>>> moving
>>>>>>>>> forward since from what we discovered working on it the
>>>>>>>>> conclusion is
>>>>>>>>>
>>>>>>>> that
>>>>>>>>
>>>>>>>>> it would require revamping the public APIs quite a lot, and it's
>>>>>>>>> not
>>>>>>>>>
>>>>>>>> clear
>>>>>>>>
>>>>>>>>> if it is a good trade-off than asking users to call process()
>>>> instead.
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>>
>>>>>>>>>>       - on-demand commit() feature
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Damian
>>>>>>>>>>
>>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>> Hi,
>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
>>>> the
>>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for your comments. I agree that this is not the best way
>>>> to
>>>>>>>>>>>> do.
>>>>>>>>>>> A
>>>>>>>>>>>
>>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as
>>>> an
>>>>>>>>>>>> argument
>>>>>>>>>>>>
>>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>>>
>>>>>>>>>>>> flexibility
>>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>>> need ij
>>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>>
>>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>
>>>>>>>>>>>> and
>>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>>> provide
>>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>>
>>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>>
>>>>>>>>>>>>> ).
>>>>>>>>>>>>>
>>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>>
>>>>>>>>>>>> RecordContext
>>>>>>>>>>> to
>>>>>>>>>>>
>>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>>
>>>>>>>>>>>> KIP/included
>>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I am personally still not convinced, that we should add
>>>> `commit()`
>>>>>>>>>>>>> at
>>>>>>>>>>> all.
>>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>>>>>> little
>>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>
>>>>>>>>>>>>> exposed
>>>>>>>>>>>> in the DSL? Just want to understand the motivation better. Why
>>>> would
>>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>>> explicitly,
>>>>>>>>>>>>> ie,
>>>>>>>>>>> via Processor API.
>>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>> add
>>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>>>> Requesting
>>>>>>>>>>>> a commit is something quite different. Additionally, a commit
>>>>>>>>>>>> does
>>>>>>>>>>>>> not
>>>>>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>>>>>>>>>> record.
>>>>>>>>>>>> To me, this does not seem to be a sound API design if we follow
>>>> this
>>>>>>>>>>>>> path.
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
>>>> to
>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> changed
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> main
>>>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is
>>>>>>>>>>>> that,
>>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>>> interfaces.
>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> if
>>>>>>>>>>>> user
>>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> KIP-159):
>>>>>>>>>>> @Override
>>>>>>>>>>>>>>>        public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>            recordContext = new RecordContext()
>>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>                public void commit() {
>>>>>>>>>>>>>>>                    context().commit();
>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>                public long offset() {
>>>>>>>>>>>>>>>                    return context().recordContext().offset();
>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>                public long timestamp() {
>>>>>>>>>>>>>>>                    return
>>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>                public String topic() {
>>>>>>>>>>>>>>>                    return context().recordContext().topic();
>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>                @Override
>>>>>>>>>>>>>>>                public int partition() {
>>>>>>>>>>>>>>>                    return
>>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>>                }
>>>>>>>>>>>>>>>          };
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
>>>> case
>>>>>>>>>>>>>> IMO.
>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>>> so
>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> only
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
>>>> amount
>>>>>>>>>>>>>> of
>>>>>>>>>>> methods and it is logical to enable inheritance between those
>>>>>>>>>>> two.
>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>> below 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 has gone though
>>>> many
>>>>>>>>>>>>>> changes
>>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>>> ProcessorContext,
>>>> to
>>>>>>>>>>>>>>> enforce
>>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> changed
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>> snippet
>>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -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
>>>>>>>>>>>> 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/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. Sax <
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 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 DSL users should not worry about
>>>>>>>>>>>>>>>>>> -- 
>>>>>>>>>>> and
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>        VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>                 final K key,
>>>>>>>>>>>>>>>>>>>>                 final RecordContext recordContext);
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>>        VR apply(final V value,
>>>>>>>>>>>>>>>>>>>>                 final K key,
>>>>>>>>>>>>>>>>>>>>                 final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>>                 final ProcessorContext processorContext);
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>>> that
>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>        VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>> pairing
>>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>>                                       final Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>                                       final Serde<V>
>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>>>>>>                                     final
>>>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>>                                     final Serde<K>
>>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>>                                     final Serde<V>
>>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
>>>> for
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> combo
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>>>>>>>>>>>>>>>>>>> adder /
>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>>>>>>>>>>>>>>>>>> functions
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who
>>>> only
>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations.
>>>> For
>>>>>>>>>>>>>>>>>>>>>> example:
>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>>>>>>                                     final
>>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>>     V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>>>>>>                                     final
>>>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> o.a.k.processor.internals
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> maintenance
>>>>>>>>>>>> (both
>>>>>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>>>>>>>>>>>>> methods),  I
>>>>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext
>>>> and
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>> I move commit() method to parent class
>>>>>>>>>>>>>>>>>>>>>> (RecordContext).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>>        VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8,
>>>> users
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> care about the key but only the context, or vice
>>>>>>>>>>>>>>>>>>>> versa, is
>>>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than
>>>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking
>>>> we'd
>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>> to
>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>> all
>>>> -- 
>>>> -- Guozhang
>>>>


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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
From a DSL point of view, users only see the new value on a
KTable#filter anyway. So why should it be an issue that we use
<newValue,oldValue> pair under the hood?

User sees newValue and gets corresponding RecordContext. I can't see any
issue here?

I cannot follow here:

>> Even when we have a statefull operation last. We move it to the very
>> first processor (KtableSource)
>> and therefore cant present a proper RecordContext.



With regard to `builder.table().filter()`:

I see you point that it would be good to be able to apply the filter()
first to reduce the stat store size of the table. But how is this
related to KIP-159?

Btw: with KIP-182, I am wondering if this would not be possible, by
putting a custom dummy store into the table and materialize the filter
result afterwards? It's not a nice way to do, but seems to be possible.


-Matthias

On 11/23/17 4:56 AM, Jan Filipiak wrote:
> The comment is valid. It falls exactly into this topic, it has exactly
> todo with this!
> Even when we have a statefull operation last. We move it to the very
> first processor (KtableSource)
> and therefore cant present a proper RecordContext.
> 
> Regarding the other Jiras you are referring to. They harm the project
> more than they do good!
> There is no need for this kind of optimizer and meta representation and
> what not. I hope they
> never get implemented.
> 
> Best Jan
> 
> 
> On 22.11.2017 14:44, Damian Guy wrote:
>> Jan, i think you comment with respect to filtering is valid, though
>> not for
>> this KIP. We have separate JIRAs for topology optimization of which this
>> falls into.
>>
>> Thanks,
>> Damian
>>
>> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Jan,
>>>
>>> Not sure I understand your argument that "we still going to present
>>> change.oldValue to the filter even though the record context() is for
>>> change.newValue". Are you referring to `KTableFilter#process()`? If yes
>>> could you point to me which LOC are you concerning about?
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> a remark of mine that got missed during migration:
>>>>
>>>> There is this problem that even though we have source.table.filter.join
>>>> the state-fullness happens at the table step not a the join step. In a
>>>> filter
>>>> we still going to present change.oldValue to the filter even though the
>>>> record context() is for change.newValue. I would go as far as applying
>>>> the filter before the table processor. Not to just get KIP-159, but
>>> because
>>>> I think its a side effect of a non ideal topology layout. If i can
>>>> filter
>>>> 99% of my
>>>> records. my state could be way smaller. Also widely escalates the
>>>> context
>>>> of the KIP
>>>>
>>>> I can only see upsides of executing the filter first.
>>>>
>>>> Best Jan
>>>>
>>>>
>>>>
>>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>>
>>>>> I am moving this back to the DISCUSS thread... Last 10 emails were
>>>>> sent
>>>>> to VOTE thread.
>>>>>
>>>>> Copying Guozhang's last summary below. Thanks for this summary. Very
>>>>> comprehensive!
>>>>>
>>>>> It seems, we all agree, that the current implementation of the context
>>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>>
>>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>>> carrying the context to more operators than just sources. It also
>>>>> seems,
>>>>> that we all agree, that many-to-one operations void the context.
>>>>>
>>>>> I still think, that just going with plain (2) is too restrictive --
>>>>> but
>>>>> I am also fine if we don't go with the full proposal of (3).
>>>>>
>>>>> Also note, that the two operators filter() and filterNot() don't
>>>>> modify
>>>>> the record and thus for both, it would be absolutely valid to keep the
>>>>> context.
>>>>>
>>>>> I personally would keep the context for at least all one-to-one
>>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>>> context further: at least the offset information is questionable for
>>>>> this case -- note thought, that semantically, the timestamp is
>>>>> inherited
>>>>> via one-to-many, and I also think this applies to "topic" and
>>>>> "partition". Thus, I think it's still valuable information we can
>>>>> carry
>>>>> downstreams.
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> Jan: which approach are you referring to as "the approach that is
>>>>> on the
>>>>>> table would be perfect"?
>>>>>>
>>>>>> Note that in today's PAPI layer we are already effectively
>>>>>> exposing the
>>>>>> record context which has the issues that we have been discussing
>>>>>> right
>>>>>> now,
>>>>>> and its semantics is always referring to the "processing record" at
>>> hand.
>>>>>> More specifically, we can think of processing a record a bit
>>>>>> different:
>>>>>>
>>>>>> 1) the record traversed the topology from source to sink, it may be
>>>>>> transformed into new object or even generate multiple new objects
>>> (think:
>>>>>> branch) along the traversal. And the record context is referring to
>>> this
>>>>>> processing record. Here the "lifetime" of the record lasts for the
>>> entire
>>>>>> topology traversal and any new records of this traversal is
>>>>>> treated as
>>>>>> different transformed values of this record (this applies to join and
>>>>>> aggregations as well).
>>>>>>
>>>>>> 2) the record being processed is wiped out in the first operator
>>>>>> after
>>>>>> the
>>>>>> source, and NEW records are forwarded to downstream operators. I.e.
>>> each
>>>>>> record only lives between two adjacent operators, once it reached the
>>> new
>>>>>> operator it's lifetime has ended and new records are generated.
>>>>>>
>>>>>> I think in the past we have talked about Streams under both context,
>>> and
>>>>>> we
>>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>>> understandable for users as it does not leak any internal
>>> implementation
>>>>>> details (e.g. for stream-table joins, table record's traversal
>>>>>> ends at
>>>>>> the
>>>>>> join operator as it is only be materialized, while stream record's
>>>>>> traversal goes through the join operator to further down until
>>>>>> sinks).
>>>>>> However if we are going to interpret following 2) above then even for
>>>>>> non-stateful operators we would not inherit record context. What
>>>>>> we're
>>>>>> discussing now, seems to infer a third semantics:
>>>>>>
>>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>>> operators,
>>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>>> "mapValues"
>>>>>>    ) and will "end" at many-to-one (stateful) operators where NEW
>>> records
>>>>>> will be generated and forwarded to the downstream operators.
>>>>>>
>>>>>> Just wanted to lay the ground for discussions so we are all on the
>>>>>> same
>>>>>> page before chatting more.
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>
>>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>>
>>>>>> Hi Matthias,
>>>>>>
>>>>>> Thanks a lot for correcting. It is a leftover from the past designs
>>> when
>>>>>> punctuate() was not deprecated.
>>>>>> I corrected.
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax
>>>>>> <ma...@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>> I just re-read the KIP.
>>>>>>> One minor comment: we don't need to introduce any deprecated
>>>>>>> methods.
>>>>>>> Thus, RichValueTransformer#punctuate can be removed completely
>>>>>>> instead
>>>>>>> of introducing it as deprecated.
>>>>>>>
>>>>>>> Otherwise looks good to me.
>>>>>>>
>>>>>>> Thanks for being so patient!
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>>
>>>>>>>> Jeyhun,
>>>>>>>>
>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
>>> think
>>>>>>>> carefully if we should add this functionality to the DSL layer
>>>>>>>> moving
>>>>>>>> forward since from what we discovered working on it the
>>>>>>>> conclusion is
>>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>>> it would require revamping the public APIs quite a lot, and it's
>>>>>>>> not
>>>>>>>>
>>>>>>> clear
>>>>>>>
>>>>>>>> if it is a good trade-off than asking users to call process()
>>> instead.
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>>> Do we need to remove the line that says:
>>>>>>>>>
>>>>>>>>>      - on-demand commit() feature
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>>>>>>>>>
>>>>>>>> wrote:
>>>>>>>> Hi,
>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
>>> the
>>>>>>>>>> overall design of KIP a lot.
>>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>>> matthias@confluent.io
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>>>> but also we need a commit() method
>>>>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for your comments. I agree that this is not the best way
>>> to
>>>>>>>>>>> do.
>>>>>>>>>> A
>>>>>>>>>>
>>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as
>>> an
>>>>>>>>>>> argument
>>>>>>>>>>>
>>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>>
>>>>>>>>>>> flexibility
>>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>>>>>>> information and not Record level info. The only thing we
>>>>>>>>>>>> need ij
>>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>>
>>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>>> timestamp
>>>>>>>>>>>>
>>>>>>>>>>> and
>>>>>>>>>>> etc) but also we need a commit() method ( we dont want to
>>>>>>>>>>> provide
>>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>>
>>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>>
>>>>>>>>>>>> ).
>>>>>>>>>>>>
>>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>>
>>>>>>>>>>> RecordContext
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>>
>>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>>
>>>>>>>>>>> KIP/included
>>>>>>>>>> jiras please let me know.
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I am personally still not convinced, that we should add
>>> `commit()`
>>>>>>>>>>>> at
>>>>>>>>>> all.
>>>>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>>>>> little
>>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should
>>>>>>>>>>>>> not be
>>>>>>>>>>>>>
>>>>>>>>>>>> exposed
>>>>>>>>>>> in the DSL? Just want to understand the motivation better. Why
>>> would
>>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To
>>>>>>>>>>>>> me,
>>>>>>>>>>>>> requesting commits is useful if you manipulated state
>>> explicitly,
>>>>>>>>>>>> ie,
>>>>>>>>>> via Processor API.
>>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me,
>>>>>>>>>>>>> that we
>>>>>>>>>>>>> add
>>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>>
>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>>> Requesting
>>>>>>>>>>> a commit is something quite different. Additionally, a commit
>>>>>>>>>>> does
>>>>>>>>>>>> not
>>>>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>>>>>>>>> record.
>>>>>>>>>>> To me, this does not seem to be a sound API design if we follow
>>> this
>>>>>>>>>>>> path.
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>>
>>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
>>> to
>>>>>>>>>>>>> enforce
>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> implementation
>>>>>>>>>>>> of
>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> changed
>>>>>>>>>>> to
>>>>>>>>>>>
>>>>>>>>>>>> this call.
>>>>>>>>>>>>>> - I think we should not deprecate
>>>>>>>>>>>>>> `ProcessorContext.commit()`.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>
>>>>>>>>>>>>> main
>>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is
>>>>>>>>>>> that,
>>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>>> interfaces.
>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>
>>>>>>>>>>>>> if
>>>>>>>>>>> user
>>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>>
>>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>>> to
>>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>>>>
>>>>>>>>>>>>> KIP-159):
>>>>>>>>>> @Override
>>>>>>>>>>>>>>       public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>           recordContext = new RecordContext()
>>>>>>>>>>>>>> {               //
>>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>>               public void commit() {
>>>>>>>>>>>>>>                   context().commit();
>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>>               public long offset() {
>>>>>>>>>>>>>>                   return context().recordContext().offset();
>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>>               public long timestamp() {
>>>>>>>>>>>>>>                   return
>>>>>>>>>>>>>> context().recordContext().timestamp();
>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>>               public String topic() {
>>>>>>>>>>>>>>                   return context().recordContext().topic();
>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>>               public int partition() {
>>>>>>>>>>>>>>                   return
>>>>>>>>>>>>>> context().recordContext().partition();
>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>         };
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
>>> case
>>>>>>>>>>>>> IMO.
>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>>> so
>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>>> `ProcessorRecordContext`. 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
>>>>>>>>>>>>>> `RecordContext()`
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>
>>>>>>>>>>>>> only
>>>>>>>>>>>> to
>>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent
>>>>>>>>>>>>>> class of
>>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
>>> amount
>>>>>>>>>>>>> of
>>>>>>>>>> methods and it is logical to enable inheritance between those
>>>>>>>>>> two.
>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> snippet
>>>>>>>>>>> below 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 has gone though
>>> many
>>>>>>>>>>>>> changes
>>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>>> with
>>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in
>>>>>>>>>>>>>>> ProcessorContext,
>>> to
>>>>>>>>>>>>>> enforce
>>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> implementation
>>>>>>>>>>>> of
>>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> changed
>>>>>>>>>>> to
>>>>>>>>>>>
>>>>>>>>>>>> this call.
>>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> snippet
>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -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
>>>>>>>>>>> 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/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. Sax <
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 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 DSL users should not worry about
>>>>>>>>>>>>>>>>> -- 
>>>>>>>>>> and
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>>> dummy
>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>> to
>>>>>>>>>>>
>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>       VR apply(final V value,
>>>>>>>>>>>>>>>>>>>                final K key,
>>>>>>>>>>>>>>>>>>>                final RecordContext recordContext);
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>>       VR apply(final V value,
>>>>>>>>>>>>>>>>>>>                final K key,
>>>>>>>>>>>>>>>>>>>                final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>>                final ProcessorContext processorContext);
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>> and
>>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>>> that
>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>       VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary
>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>> pairing
>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>> super
>>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>>                                      final Serde<KR>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>                                      final Serde<V>
>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>>>>>                                    final
>>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>
>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>>                                    final Serde<K>
>>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>>                                    final Serde<V>
>>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
>>> for
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> combo
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>>>>>>>>>>>>>>>>>> adder /
>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>>>>>>>>>>>>>>>>> functions
>>>>>>>>>>> for
>>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who
>>> only
>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>> to
>>>>>>>>>>>
>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations.
>>> For
>>>>>>>>>>>>>>>>>>>>> example:
>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>>>>>                                    final
>>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>>    V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>>>>>                                    final
>>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>
>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> o.a.k.processor.internals
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> maintenance
>>>>>>>>>>> (both
>>>>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>>>>>>>>>>>> methods),  I
>>>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext
>>> and
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>> I move commit() method to parent class
>>>>>>>>>>>>>>>>>>>>> (RecordContext).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>>       VR apply(final V1 value1, final V2 value2,
>>>>>>>>>>>>>>>>>>>>>> final K
>>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8,
>>> users
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>> would
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> care about the key but only the context, or vice
>>>>>>>>>>>>>>>>>>> versa, is
>>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than
>>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>> dummy
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking
>>> we'd
>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>> to
>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>
>>>
>>> -- 
>>> -- Guozhang
>>>
> 


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

Posted by Jan Filipiak <Ja...@trivago.com>.
The comment is valid. It falls exactly into this topic, it has exactly 
todo with this!
Even when we have a statefull operation last. We move it to the very 
first processor (KtableSource)
and therefore cant present a proper RecordContext.

Regarding the other Jiras you are referring to. They harm the project 
more than they do good!
There is no need for this kind of optimizer and meta representation and 
what not. I hope they
never get implemented.

Best Jan


On 22.11.2017 14:44, Damian Guy wrote:
> Jan, i think you comment with respect to filtering is valid, though not for
> this KIP. We have separate JIRAs for topology optimization of which this
> falls into.
>
> Thanks,
> Damian
>
> On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:
>
>> Jan,
>>
>> Not sure I understand your argument that "we still going to present
>> change.oldValue to the filter even though the record context() is for
>> change.newValue". Are you referring to `KTableFilter#process()`? If yes
>> could you point to me which LOC are you concerning about?
>>
>>
>> Guozhang
>>
>>
>> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>>> a remark of mine that got missed during migration:
>>>
>>> There is this problem that even though we have source.table.filter.join
>>> the state-fullness happens at the table step not a the join step. In a
>>> filter
>>> we still going to present change.oldValue to the filter even though the
>>> record context() is for change.newValue. I would go as far as applying
>>> the filter before the table processor. Not to just get KIP-159, but
>> because
>>> I think its a side effect of a non ideal topology layout. If i can filter
>>> 99% of my
>>> records. my state could be way smaller. Also widely escalates the context
>>> of the KIP
>>>
>>> I can only see upsides of executing the filter first.
>>>
>>> Best Jan
>>>
>>>
>>>
>>> On 20.11.2017 22:22, Matthias J. Sax wrote:
>>>
>>>> I am moving this back to the DISCUSS thread... Last 10 emails were sent
>>>> to VOTE thread.
>>>>
>>>> Copying Guozhang's last summary below. Thanks for this summary. Very
>>>> comprehensive!
>>>>
>>>> It seems, we all agree, that the current implementation of the context
>>>> at PAPI level is ok, but we should not leak it into DSL.
>>>>
>>>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>>>> carrying the context to more operators than just sources. It also seems,
>>>> that we all agree, that many-to-one operations void the context.
>>>>
>>>> I still think, that just going with plain (2) is too restrictive -- but
>>>> I am also fine if we don't go with the full proposal of (3).
>>>>
>>>> Also note, that the two operators filter() and filterNot() don't modify
>>>> the record and thus for both, it would be absolutely valid to keep the
>>>> context.
>>>>
>>>> I personally would keep the context for at least all one-to-one
>>>> operators. One-to-many is debatable and I am fine to not carry the
>>>> context further: at least the offset information is questionable for
>>>> this case -- note thought, that semantically, the timestamp is inherited
>>>> via one-to-many, and I also think this applies to "topic" and
>>>> "partition". Thus, I think it's still valuable information we can carry
>>>> downstreams.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> Jan: which approach are you referring to as "the approach that is on the
>>>>> table would be perfect"?
>>>>>
>>>>> Note that in today's PAPI layer we are already effectively exposing the
>>>>> record context which has the issues that we have been discussing right
>>>>> now,
>>>>> and its semantics is always referring to the "processing record" at
>> hand.
>>>>> More specifically, we can think of processing a record a bit different:
>>>>>
>>>>> 1) the record traversed the topology from source to sink, it may be
>>>>> transformed into new object or even generate multiple new objects
>> (think:
>>>>> branch) along the traversal. And the record context is referring to
>> this
>>>>> processing record. Here the "lifetime" of the record lasts for the
>> entire
>>>>> topology traversal and any new records of this traversal is treated as
>>>>> different transformed values of this record (this applies to join and
>>>>> aggregations as well).
>>>>>
>>>>> 2) the record being processed is wiped out in the first operator after
>>>>> the
>>>>> source, and NEW records are forwarded to downstream operators. I.e.
>> each
>>>>> record only lives between two adjacent operators, once it reached the
>> new
>>>>> operator it's lifetime has ended and new records are generated.
>>>>>
>>>>> I think in the past we have talked about Streams under both context,
>> and
>>>>> we
>>>>> do not have a clear agreement. I agree that 2) is logically more
>>>>> understandable for users as it does not leak any internal
>> implementation
>>>>> details (e.g. for stream-table joins, table record's traversal ends at
>>>>> the
>>>>> join operator as it is only be materialized, while stream record's
>>>>> traversal goes through the join operator to further down until sinks).
>>>>> However if we are going to interpret following 2) above then even for
>>>>> non-stateful operators we would not inherit record context. What we're
>>>>> discussing now, seems to infer a third semantics:
>>>>>
>>>>> 3) a record would traverse "through" one-to-one (non-stateful)
>> operators,
>>>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>>>> "mapValues"
>>>>>    ) and will "end" at many-to-one (stateful) operators where NEW
>> records
>>>>> will be generated and forwarded to the downstream operators.
>>>>>
>>>>> Just wanted to lay the ground for discussions so we are all on the same
>>>>> page before chatting more.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>
>>>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>>>
>>>>> Hi Matthias,
>>>>>
>>>>> Thanks a lot for correcting. It is a leftover from the past designs
>> when
>>>>> punctuate() was not deprecated.
>>>>> I corrected.
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>
>>>>> I just re-read the KIP.
>>>>>> One minor comment: we don't need to introduce any deprecated methods.
>>>>>> Thus, RichValueTransformer#punctuate can be removed completely instead
>>>>>> of introducing it as deprecated.
>>>>>>
>>>>>> Otherwise looks good to me.
>>>>>>
>>>>>> Thanks for being so patient!
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>>>
>>>>>>> Jeyhun,
>>>>>>>
>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
>> think
>>>>>>> carefully if we should add this functionality to the DSL layer moving
>>>>>>> forward since from what we discovered working on it the conclusion is
>>>>>>>
>>>>>> that
>>>>>>
>>>>>>> it would require revamping the public APIs quite a lot, and it's not
>>>>>>>
>>>>>> clear
>>>>>>
>>>>>>> if it is a good trade-off than asking users to call process()
>> instead.
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Jeyhun, thanks, looks good.
>>>>>>>> Do we need to remove the line that says:
>>>>>>>>
>>>>>>>>      - on-demand commit() feature
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>>>>>>>>
>>>>>>> wrote:
>>>>>>> Hi,
>>>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
>> the
>>>>>>>>> overall design of KIP a lot.
>>>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Jeyhun
>>>>>>>>>
>>>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>>>> matthias@confluent.io
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>>> but also we need a commit() method
>>>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Matthias,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your comments. I agree that this is not the best way
>> to
>>>>>>>>>> do.
>>>>>>>>> A
>>>>>>>>>
>>>>>>>>>> bit of history behind this design.
>>>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as
>> an
>>>>>>>>>> argument
>>>>>>>>>>
>>>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>>>
>>>>>>>>>> flexibility
>>>>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>>>>>> information and not Record level info. The only thing we need ij
>>>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>>>
>>>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>>>> timestamp
>>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>> etc) but also we need a commit() method ( we dont want to provide
>>>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>>>
>>>>>>>>>> ProcessorContext.commit()
>>>>>>>>>>
>>>>>>>>>>> ).
>>>>>>>>>>>
>>>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>>>
>>>>>>>>>> RecordContext
>>>>>>>>> to
>>>>>>>>>
>>>>>>>>>> ProcessorContext() .
>>>>>>>>>>>
>>>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>>>
>>>>>>>>>> KIP/included
>>>>>>>>> jiras please let me know.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> I am personally still not convinced, that we should add
>> `commit()`
>>>>>>>>>>> at
>>>>>>>>> all.
>>>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>>>> little
>>>>>>>>>>>> bit? Isn't requesting commits a low level API that should not be
>>>>>>>>>>>>
>>>>>>>>>>> exposed
>>>>>>>>>> in the DSL? Just want to understand the motivation better. Why
>> would
>>>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To me,
>>>>>>>>>>>> requesting commits is useful if you manipulated state
>> explicitly,
>>>>>>>>>>> ie,
>>>>>>>>> via Processor API.
>>>>>>>>>>>> Also, for the solution: it seem rather unnatural to me, that we
>>>>>>>>>>>> add
>>>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>>>
>>>>>>>>>>> `RecordContext`
>>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>> Requesting
>>>>>>>>>> a commit is something quite different. Additionally, a commit does
>>>>>>>>>>> not
>>>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>>>>>>>> record.
>>>>>>>>>> To me, this does not seem to be a sound API design if we follow
>> this
>>>>>>>>>>> path.
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>>>
>>>>>>>>>>>> misunderstanding.
>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
>> to
>>>>>>>>>>>> enforce
>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>
>>>>>>>>>>>>> implementation
>>>>>>>>>>> of
>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>>>>>>>>>>
>>>>>>>>>>>>> changed
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> this call.
>>>>>>>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`.
>>>>>>>>>>>>> The
>>>>>>>>>>>>>
>>>>>>>>>>>> main
>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>>>>>>>>>>>> `RecordContext` is the one which is provided in Rich
>> interfaces.
>>>>>>>>>>>>> So
>>>>>>>>>>>>>
>>>>>>>>>>>> if
>>>>>>>>>> user
>>>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>>>
>>>>>>>>>>>> `RecordContext`
>>>>>>>>>>> to
>>>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>>>
>>>>>>>>>>>> KIP-159):
>>>>>>>>> @Override
>>>>>>>>>>>>>       public void process(final K1 key, final V1 value) {
>>>>>>>>>>>>>
>>>>>>>>>>>>>           recordContext = new RecordContext() {               //
>>>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>               public void commit() {
>>>>>>>>>>>>>                   context().commit();
>>>>>>>>>>>>>               }
>>>>>>>>>>>>>
>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>               public long offset() {
>>>>>>>>>>>>>                   return context().recordContext().offset();
>>>>>>>>>>>>>               }
>>>>>>>>>>>>>
>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>               public long timestamp() {
>>>>>>>>>>>>>                   return context().recordContext().timestamp();
>>>>>>>>>>>>>               }
>>>>>>>>>>>>>
>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>               public String topic() {
>>>>>>>>>>>>>                   return context().recordContext().topic();
>>>>>>>>>>>>>               }
>>>>>>>>>>>>>
>>>>>>>>>>>>>               @Override
>>>>>>>>>>>>>               public int partition() {
>>>>>>>>>>>>>                   return context().recordContext().partition();
>>>>>>>>>>>>>               }
>>>>>>>>>>>>>         };
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
>> case
>>>>>>>>>>>> IMO.
>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>
>>>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>>>> so
>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()`
>>>>>>>>>>>>> just
>>>>>>>>>>>>>
>>>>>>>>>>>> only
>>>>>>>>>>> to
>>>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
>>>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
>> amount
>>>>>>>>>>>> of
>>>>>>>>> methods and it is logical to enable inheritance between those two.
>>>>>>>>>>>>> 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
>>>>>>>>>>>>> snippet
>>>>>>>>>> below 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 has gone though
>> many
>>>>>>>>>>>> changes
>>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>> KAFKA-3907
>>>>>>>>>> with
>>>>>>>>>>>>> this 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:
>>>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
>> to
>>>>>>>>>>>>> enforce
>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>>>
>>>>>>>>>>>>> implementation
>>>>>>>>>>> of
>>>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>>>>>>>>>>
>>>>>>>>>>>>> changed
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> this call.
>>>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>>> 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
>>>>>>>>>>>>> snippet
>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>>>
>>>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -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
>>>>>>>>>> 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/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. Sax <
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 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 DSL users should not worry about
>>>>>>>>>>>>>>>> --
>>>>>>>>> and
>>>>>>>>>>> if
>>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>> dummy
>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>> argument
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>       VR apply(final V value,
>>>>>>>>>>>>>>>>>>                final K key,
>>>>>>>>>>>>>>>>>>                final RecordContext recordContext);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>>>       VR apply(final V value,
>>>>>>>>>>>>>>>>>>                final K key,
>>>>>>>>>>>>>>>>>>                final RecordContext recordContext,
>>>>>>>>>>>>>>>>>>                final ProcessorContext processorContext);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>> and
>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
>> that
>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>       VR apply(final V1 value1, final V2 value2, final K
>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> their
>>>>>>>>> pairing
>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>> super
>>>>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>>>                                      final Serde<KR>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>                                      final Serde<V>
>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>>>>                                    final
>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>> K,
>>>>>>>>>>
>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>>>                                    final Serde<K>
>>>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>>>                                    final Serde<V>
>>>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
>> for
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> combo
>>>>>>>>>>> of
>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>>>>>>>>>>>>>>>>> adder /
>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>>>>>>>>>>>>>>>> functions
>>>>>>>>>> for
>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who
>> only
>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations.
>> For
>>>>>>>>>>>>>>>>>>>> example:
>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>>>>                                    final
>>>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>>>    V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>>>>                                    final
>> RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>> super
>>>>>>>>>> K,
>>>>>>>>>>
>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> o.a.k.processor.internals
>>>>>>>>>>> to
>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> maintenance
>>>>>>>>>> (both
>>>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>>>>>>>>>>> methods),  I
>>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext
>> and
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>>>       VR apply(final V1 value1, final V2 value2, final K
>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8,
>> users
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>> would
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>> to
>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than
>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>> dummy
>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking
>> we'd
>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>> to
>>>>>>>>>>> make
>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>
>>
>> --
>> -- Guozhang
>>


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

Posted by Damian Guy <da...@gmail.com>.
Jan, i think you comment with respect to filtering is valid, though not for
this KIP. We have separate JIRAs for topology optimization of which this
falls into.

Thanks,
Damian

On Wed, 22 Nov 2017 at 02:25 Guozhang Wang <wa...@gmail.com> wrote:

> Jan,
>
> Not sure I understand your argument that "we still going to present
> change.oldValue to the filter even though the record context() is for
> change.newValue". Are you referring to `KTableFilter#process()`? If yes
> could you point to me which LOC are you concerning about?
>
>
> Guozhang
>
>
> On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > a remark of mine that got missed during migration:
> >
> > There is this problem that even though we have source.table.filter.join
> > the state-fullness happens at the table step not a the join step. In a
> > filter
> > we still going to present change.oldValue to the filter even though the
> > record context() is for change.newValue. I would go as far as applying
> > the filter before the table processor. Not to just get KIP-159, but
> because
> > I think its a side effect of a non ideal topology layout. If i can filter
> > 99% of my
> > records. my state could be way smaller. Also widely escalates the context
> > of the KIP
> >
> > I can only see upsides of executing the filter first.
> >
> > Best Jan
> >
> >
> >
> > On 20.11.2017 22:22, Matthias J. Sax wrote:
> >
> >> I am moving this back to the DISCUSS thread... Last 10 emails were sent
> >> to VOTE thread.
> >>
> >> Copying Guozhang's last summary below. Thanks for this summary. Very
> >> comprehensive!
> >>
> >> It seems, we all agree, that the current implementation of the context
> >> at PAPI level is ok, but we should not leak it into DSL.
> >>
> >> Thus, we can go with (2) or (3), were (3) is an extension to (2)
> >> carrying the context to more operators than just sources. It also seems,
> >> that we all agree, that many-to-one operations void the context.
> >>
> >> I still think, that just going with plain (2) is too restrictive -- but
> >> I am also fine if we don't go with the full proposal of (3).
> >>
> >> Also note, that the two operators filter() and filterNot() don't modify
> >> the record and thus for both, it would be absolutely valid to keep the
> >> context.
> >>
> >> I personally would keep the context for at least all one-to-one
> >> operators. One-to-many is debatable and I am fine to not carry the
> >> context further: at least the offset information is questionable for
> >> this case -- note thought, that semantically, the timestamp is inherited
> >> via one-to-many, and I also think this applies to "topic" and
> >> "partition". Thus, I think it's still valuable information we can carry
> >> downstreams.
> >>
> >>
> >> -Matthias
> >>
> >> Jan: which approach are you referring to as "the approach that is on the
> >>> table would be perfect"?
> >>>
> >>> Note that in today's PAPI layer we are already effectively exposing the
> >>> record context which has the issues that we have been discussing right
> >>> now,
> >>> and its semantics is always referring to the "processing record" at
> hand.
> >>> More specifically, we can think of processing a record a bit different:
> >>>
> >>> 1) the record traversed the topology from source to sink, it may be
> >>> transformed into new object or even generate multiple new objects
> (think:
> >>> branch) along the traversal. And the record context is referring to
> this
> >>> processing record. Here the "lifetime" of the record lasts for the
> entire
> >>> topology traversal and any new records of this traversal is treated as
> >>> different transformed values of this record (this applies to join and
> >>> aggregations as well).
> >>>
> >>> 2) the record being processed is wiped out in the first operator after
> >>> the
> >>> source, and NEW records are forwarded to downstream operators. I.e.
> each
> >>> record only lives between two adjacent operators, once it reached the
> new
> >>> operator it's lifetime has ended and new records are generated.
> >>>
> >>> I think in the past we have talked about Streams under both context,
> and
> >>> we
> >>> do not have a clear agreement. I agree that 2) is logically more
> >>> understandable for users as it does not leak any internal
> implementation
> >>> details (e.g. for stream-table joins, table record's traversal ends at
> >>> the
> >>> join operator as it is only be materialized, while stream record's
> >>> traversal goes through the join operator to further down until sinks).
> >>> However if we are going to interpret following 2) above then even for
> >>> non-stateful operators we would not inherit record context. What we're
> >>> discussing now, seems to infer a third semantics:
> >>>
> >>> 3) a record would traverse "through" one-to-one (non-stateful)
> operators,
> >>> will "replicate" at one-to-many (non-stateful) operators (think:
> >>> "mapValues"
> >>>   ) and will "end" at many-to-one (stateful) operators where NEW
> records
> >>> will be generated and forwarded to the downstream operators.
> >>>
> >>> Just wanted to lay the ground for discussions so we are all on the same
> >>> page before chatting more.
> >>>
> >>>
> >>> Guozhang
> >>>
> >>
> >>
> >> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
> >>
> >>> Hi Matthias,
> >>>
> >>> Thanks a lot for correcting. It is a leftover from the past designs
> when
> >>> punctuate() was not deprecated.
> >>> I corrected.
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
> >>> wrote:
> >>>
> >>> I just re-read the KIP.
> >>>>
> >>>> One minor comment: we don't need to introduce any deprecated methods.
> >>>> Thus, RichValueTransformer#punctuate can be removed completely instead
> >>>> of introducing it as deprecated.
> >>>>
> >>>> Otherwise looks good to me.
> >>>>
> >>>> Thanks for being so patient!
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
> >>>>
> >>>>> Jeyhun,
> >>>>>
> >>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should
> think
> >>>>> carefully if we should add this functionality to the DSL layer moving
> >>>>> forward since from what we discovered working on it the conclusion is
> >>>>>
> >>>> that
> >>>>
> >>>>> it would require revamping the public APIs quite a lot, and it's not
> >>>>>
> >>>> clear
> >>>>
> >>>>> if it is a good trade-off than asking users to call process()
> instead.
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>> Hi Jeyhun, thanks, looks good.
> >>>>>> Do we need to remove the line that says:
> >>>>>>
> >>>>>>     - on-demand commit() feature
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Damian
> >>>>>>
> >>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
> >>>>>>
> >>>>> wrote:
> >>>>
> >>>>> Hi,
> >>>>>>>
> >>>>>>> I removed the 'commit()' feature, as we discussed. It simplified
> the
> >>>>>>> overall design of KIP a lot.
> >>>>>>> If it is ok, I would like to start a VOTE thread.
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Jeyhun
> >>>>>>>
> >>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
> >>>>>>> matthias@confluent.io
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>> Thanks. I understand what you are saying, but I don't agree that
> >>>>>>>>
> >>>>>>>> but also we need a commit() method
> >>>>>>>>>
> >>>>>>>> I would just not provide `commit()` at DSL level and close the
> >>>>>>>> corresponding Jira as "not a problem" or similar.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> >>>>>>>>
> >>>>>>>>> Hi Matthias,
> >>>>>>>>>
> >>>>>>>>> Thanks for your comments. I agree that this is not the best way
> to
> >>>>>>>>>
> >>>>>>>> do.
> >>>>>>
> >>>>>>> A
> >>>>>>>
> >>>>>>>> bit of history behind this design.
> >>>>>>>>>
> >>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as
> an
> >>>>>>>>>
> >>>>>>>> argument
> >>>>>>>>
> >>>>>>>>> in Rich interfaces. However, we dont want to give users that
> >>>>>>>>>
> >>>>>>>> flexibility
> >>>>>>>
> >>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
> >>>>>>>>> information and not Record level info. The only thing we need ij
> >>>>>>>>> ProcessorContext is commit() method.
> >>>>>>>>>
> >>>>>>>>> So, as far as I understood, we need recor context (offset,
> >>>>>>>>> timestamp
> >>>>>>>>>
> >>>>>>>> and
> >>>>>>>
> >>>>>>>> etc) but also we need a commit() method ( we dont want to provide
> >>>>>>>>> ProcessorContext as a parameter so users can use
> >>>>>>>>>
> >>>>>>>> ProcessorContext.commit()
> >>>>>>>>
> >>>>>>>>> ).
> >>>>>>>>>
> >>>>>>>>> As a result, I thought to “propagate” commit() call from
> >>>>>>>>>
> >>>>>>>> RecordContext
> >>>>>>
> >>>>>>> to
> >>>>>>>
> >>>>>>>> ProcessorContext() .
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> If there is a misunderstanding in motvation/discussion of
> >>>>>>>>>
> >>>>>>>> KIP/included
> >>>>>>
> >>>>>>> jiras please let me know.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Cheers,
> >>>>>>>>> Jeyhun
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
> >>>>>>>>> matthias@confluent.io
> >>>>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> I am personally still not convinced, that we should add
> `commit()`
> >>>>>>>>>>
> >>>>>>>>> at
> >>>>>>
> >>>>>>> all.
> >>>>>>>>
> >>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
> >>>>>>>>>> little
> >>>>>>>>>> bit? Isn't requesting commits a low level API that should not be
> >>>>>>>>>>
> >>>>>>>>> exposed
> >>>>>>>
> >>>>>>>> in the DSL? Just want to understand the motivation better. Why
> would
> >>>>>>>>>> anybody that uses the DSL ever want to request a commit? To me,
> >>>>>>>>>> requesting commits is useful if you manipulated state
> explicitly,
> >>>>>>>>>>
> >>>>>>>>> ie,
> >>>>>>
> >>>>>>> via Processor API.
> >>>>>>>>>>
> >>>>>>>>>> Also, for the solution: it seem rather unnatural to me, that we
> >>>>>>>>>> add
> >>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
> >>>>>>>>>>
> >>>>>>>>> `RecordContext`
> >>>>>>>
> >>>>>>>> is an helper object that provide access to record meta data.
> >>>>>>>>>>
> >>>>>>>>> Requesting
> >>>>>>>
> >>>>>>>> a commit is something quite different. Additionally, a commit does
> >>>>>>>>>>
> >>>>>>>>> not
> >>>>>>
> >>>>>>> commit a specific record but a `RecrodContext` is for a specific
> >>>>>>>>>>
> >>>>>>>>> record.
> >>>>>>>
> >>>>>>>> To me, this does not seem to be a sound API design if we follow
> this
> >>>>>>>>>>
> >>>>>>>>> path.
> >>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for your suggestions.
> >>>>>>>>>>>
> >>>>>>>>>>> I have some comments, to make sure that there is no
> >>>>>>>>>>>
> >>>>>>>>>> misunderstanding.
> >>>>>>
> >>>>>>>
> >>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
> to
> >>>>>>>>>>>
> >>>>>>>>>> enforce
> >>>>>>>>
> >>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
> >>>>>>>>>>>>
> >>>>>>>>>>> implementation
> >>>>>>>>
> >>>>>>>>> of
> >>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> >>>>>>>>>>>>
> >>>>>>>>>>> changed
> >>>>>>>
> >>>>>>>> to
> >>>>>>>>
> >>>>>>>>> this call.
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`.
> >>>>>>>>>>> The
> >>>>>>>>>>>
> >>>>>>>>>> main
> >>>>>>>
> >>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
> >>>>>>>>>>> `RecordContext` is the one which is provided in Rich
> interfaces.
> >>>>>>>>>>> So
> >>>>>>>>>>>
> >>>>>>>>>> if
> >>>>>>>
> >>>>>>>> user
> >>>>>>>>>>
> >>>>>>>>>>> wants to commit, then there should be some method inside
> >>>>>>>>>>>
> >>>>>>>>>> `RecordContext`
> >>>>>>>>
> >>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
> >>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
> >>>>>>>>>>>
> >>>>>>>>>> KIP-159):
> >>>>>>
> >>>>>>> @Override
> >>>>>>>>>>>      public void process(final K1 key, final V1 value) {
> >>>>>>>>>>>
> >>>>>>>>>>>          recordContext = new RecordContext() {               //
> >>>>>>>>>>> recordContext initialization is added in this KIP
> >>>>>>>>>>>              @Override
> >>>>>>>>>>>              public void commit() {
> >>>>>>>>>>>                  context().commit();
> >>>>>>>>>>>              }
> >>>>>>>>>>>
> >>>>>>>>>>>              @Override
> >>>>>>>>>>>              public long offset() {
> >>>>>>>>>>>                  return context().recordContext().offset();
> >>>>>>>>>>>              }
> >>>>>>>>>>>
> >>>>>>>>>>>              @Override
> >>>>>>>>>>>              public long timestamp() {
> >>>>>>>>>>>                  return context().recordContext().timestamp();
> >>>>>>>>>>>              }
> >>>>>>>>>>>
> >>>>>>>>>>>              @Override
> >>>>>>>>>>>              public String topic() {
> >>>>>>>>>>>                  return context().recordContext().topic();
> >>>>>>>>>>>              }
> >>>>>>>>>>>
> >>>>>>>>>>>              @Override
> >>>>>>>>>>>              public int partition() {
> >>>>>>>>>>>                  return context().recordContext().partition();
> >>>>>>>>>>>              }
> >>>>>>>>>>>        };
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this
> case
> >>>>>>>>>>>
> >>>>>>>>>> IMO.
> >>>>>>
> >>>>>>>
> >>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>
> >>>>>>>>>> `ProcessorRecordContext`,
> >>>>>>>>
> >>>>>>>>> so
> >>>>>>>>>>
> >>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> - Actually, I don't think that we need `commit()` in
> >>>>>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()`
> >>>>>>>>>>> just
> >>>>>>>>>>>
> >>>>>>>>>> only
> >>>>>>>>
> >>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
> >>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite
> amount
> >>>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>
> >>>>>>> methods and it is logical to enable inheritance between those two.
> >>>>>>>>>>>
> >>>>>>>>>>> 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
> >>>>>>>>>>>>
> >>>>>>>>>>> snippet
> >>>>>>>
> >>>>>>>> below 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 has gone though
> many
> >>>>>>>>>>>
> >>>>>>>>>> changes
> >>>>>>>>
> >>>>>>>>> based on user/developer needs, both in
> >>>>>>>>>>>
> >>>>>>>>>> documentation-/implementation-wise.
> >>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Jeyhun
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
> >>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
> >>>>>>>>>>>>
> >>>>>>>>>>> KAFKA-3907
> >>>>>>>
> >>>>>>>> with
> >>>>>>>>>>
> >>>>>>>>>>> this 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:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext,
> to
> >>>>>>>>>>>>
> >>>>>>>>>>> enforce
> >>>>>>>>
> >>>>>>>>> user to consolidate this call as
> >>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
> >>>>>>>>>>>>
> >>>>>>>>>>> implementation
> >>>>>>>>
> >>>>>>>>> of
> >>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> >>>>>>>>>>>>
> >>>>>>>>>>> changed
> >>>>>>>
> >>>>>>>> to
> >>>>>>>>
> >>>>>>>>> this call.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>>>>>>>>
> >>>>>>>>>>> `ProcessorRecordContext`, so
> >>>>>>>>>>
> >>>>>>>>>>> that it can implement the commit call itself.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 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
> >>>>>>>>>>>>
> >>>>>>>>>>> snippet
> >>>>>>>
> >>>>>>>> below would need to be updated as well.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> >>>>>>>>>>>>
> >>>>>>>>>>> matthias@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.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -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
> >>>>>>>
> >>>>>>>> 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/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. Sax <
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 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 DSL users should not worry about
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> --
> >>>>>>
> >>>>>>> and
> >>>>>>>>
> >>>>>>>>> if
> >>>>>>>>>>>>
> >>>>>>>>>>>>> one really needs this, an advanced user can still insert a
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> dummy
> >>>>>>
> >>>>>>> `transform` to request a commit from there.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> argument
> >>>>>>>
> >>>>>>>> to
> >>>>>>>>
> >>>>>>>>> Rich
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> methods:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>>>>>      VR apply(final V value,
> >>>>>>>>>>>>>>>>               final K key,
> >>>>>>>>>>>>>>>>               final RecordContext recordContext);
> >>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> we can adopt
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>>>>>      VR apply(final V value,
> >>>>>>>>>>>>>>>>               final K key,
> >>>>>>>>>>>>>>>>               final RecordContext recordContext,
> >>>>>>>>>>>>>>>>               final ProcessorContext processorContext);
> >>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> However, in this case, a user can get confused as
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>
> >>>>>>>> and
> >>>>>>>>>>>>
> >>>>>>>>>>>>> RecordContext share some methods with the same name.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wangguoz@gmail.com
> >>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only
> that
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> single
> >>>>>>>>
> >>>>>>>>> record
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> itself.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K
> >>>>>>>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> final
> >>>>>>>>
> >>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> - Fixed.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> their
> >>>>>>
> >>>>>>> pairing
> >>>>>>>>>>>>
> >>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
> >>>>>>
> >>>>>>> super
> >>>>>>>>>>>>
> >>>>>>>>>>>>> K,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>>>>>>>>>                                     final Serde<KR>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> keySerde,
> >>>>>>
> >>>>>>>                                     final Serde<V>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> valSerde);
> >>>>>>
> >>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>>>>>>>>>>>                                   final
> RichValueJoiner<?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> super
> >>>>>>>
> >>>>>>>> K,
> >>>>>>>>
> >>>>>>>>> ?
> >>>>>>>>>>>>
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>>>>>>>>>                                   final Serde<K>
> >>>>>>>>>>>>>>>>>>> keySerde,
> >>>>>>>>>>>>>>>>>>>                                   final Serde<V>
> >>>>>>>>>>>>>>>>>>> valSerde);
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Fixed
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs
> for
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> combo
> >>>>>>>>
> >>>>>>>>> of
> >>>>>>>>>>>>
> >>>>>>>>>>>>> both
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> adder /
> >>>>>>
> >>>>>>> subtractor,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> functions
> >>>>>>>
> >>>>>>>> for
> >>>>>>>>>>>>
> >>>>>>>>>>>>> both;
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who
> only
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> want
> >>>>>>>
> >>>>>>>> to
> >>>>>>>>
> >>>>>>>>> access
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations.
> For
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> example:
> >>>>>>>>>>
> >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> globalKTable,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>                                   final
> >>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> super
> >>>>>>>>
> >>>>>>>>> K, ?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>>>>>   V, ? extends GK> keyValueMapper,
> >>>>>>>>>>>>>>>>>>>                                   final
> RichValueJoiner<?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> super
> >>>>>>>
> >>>>>>>> K,
> >>>>>>>>
> >>>>>>>>> ?
> >>>>>>>>>>>>
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Agreed. Fixed.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
> >>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> its
> >>>>>>>
> >>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> o.a.k.processor.internals
> >>>>>>>>
> >>>>>>>>> to
> >>>>>>>>>>>>
> >>>>>>>>>>>>> o.a.k.processor.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> RecordContext?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -
> >>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> maintenance
> >>>>>>>
> >>>>>>>> (both
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> methods),  I
> >>>>>>>>>>>>
> >>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
> >>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext
> and
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> in
> >>>>>>
> >>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K
> >>>>>>>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> final
> >>>>>>>>
> >>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8,
> users
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> that
> >>>>>>
> >>>>>>> would
> >>>>>>>>>>>>
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> likely
> >>>>>>>>
> >>>>>>>>> to
> >>>>>>>>>>>>
> >>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than
> >>>>>>>>>>>>>>>>>>> putting
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>
> >>>>>>>> dummy
> >>>>>>>>>>>>
> >>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking
> we'd
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> like
> >>>>>>
> >>>>>>> to
> >>>>>>>>
> >>>>>>>>> make
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
Jan,

Not sure I understand your argument that "we still going to present
change.oldValue to the filter even though the record context() is for
change.newValue". Are you referring to `KTableFilter#process()`? If yes
could you point to me which LOC are you concerning about?


Guozhang


On Mon, Nov 20, 2017 at 9:29 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> a remark of mine that got missed during migration:
>
> There is this problem that even though we have source.table.filter.join
> the state-fullness happens at the table step not a the join step. In a
> filter
> we still going to present change.oldValue to the filter even though the
> record context() is for change.newValue. I would go as far as applying
> the filter before the table processor. Not to just get KIP-159, but because
> I think its a side effect of a non ideal topology layout. If i can filter
> 99% of my
> records. my state could be way smaller. Also widely escalates the context
> of the KIP
>
> I can only see upsides of executing the filter first.
>
> Best Jan
>
>
>
> On 20.11.2017 22:22, Matthias J. Sax wrote:
>
>> I am moving this back to the DISCUSS thread... Last 10 emails were sent
>> to VOTE thread.
>>
>> Copying Guozhang's last summary below. Thanks for this summary. Very
>> comprehensive!
>>
>> It seems, we all agree, that the current implementation of the context
>> at PAPI level is ok, but we should not leak it into DSL.
>>
>> Thus, we can go with (2) or (3), were (3) is an extension to (2)
>> carrying the context to more operators than just sources. It also seems,
>> that we all agree, that many-to-one operations void the context.
>>
>> I still think, that just going with plain (2) is too restrictive -- but
>> I am also fine if we don't go with the full proposal of (3).
>>
>> Also note, that the two operators filter() and filterNot() don't modify
>> the record and thus for both, it would be absolutely valid to keep the
>> context.
>>
>> I personally would keep the context for at least all one-to-one
>> operators. One-to-many is debatable and I am fine to not carry the
>> context further: at least the offset information is questionable for
>> this case -- note thought, that semantically, the timestamp is inherited
>> via one-to-many, and I also think this applies to "topic" and
>> "partition". Thus, I think it's still valuable information we can carry
>> downstreams.
>>
>>
>> -Matthias
>>
>> Jan: which approach are you referring to as "the approach that is on the
>>> table would be perfect"?
>>>
>>> Note that in today's PAPI layer we are already effectively exposing the
>>> record context which has the issues that we have been discussing right
>>> now,
>>> and its semantics is always referring to the "processing record" at hand.
>>> More specifically, we can think of processing a record a bit different:
>>>
>>> 1) the record traversed the topology from source to sink, it may be
>>> transformed into new object or even generate multiple new objects (think:
>>> branch) along the traversal. And the record context is referring to this
>>> processing record. Here the "lifetime" of the record lasts for the entire
>>> topology traversal and any new records of this traversal is treated as
>>> different transformed values of this record (this applies to join and
>>> aggregations as well).
>>>
>>> 2) the record being processed is wiped out in the first operator after
>>> the
>>> source, and NEW records are forwarded to downstream operators. I.e. each
>>> record only lives between two adjacent operators, once it reached the new
>>> operator it's lifetime has ended and new records are generated.
>>>
>>> I think in the past we have talked about Streams under both context, and
>>> we
>>> do not have a clear agreement. I agree that 2) is logically more
>>> understandable for users as it does not leak any internal implementation
>>> details (e.g. for stream-table joins, table record's traversal ends at
>>> the
>>> join operator as it is only be materialized, while stream record's
>>> traversal goes through the join operator to further down until sinks).
>>> However if we are going to interpret following 2) above then even for
>>> non-stateful operators we would not inherit record context. What we're
>>> discussing now, seems to infer a third semantics:
>>>
>>> 3) a record would traverse "through" one-to-one (non-stateful) operators,
>>> will "replicate" at one-to-many (non-stateful) operators (think:
>>> "mapValues"
>>>   ) and will "end" at many-to-one (stateful) operators where NEW records
>>> will be generated and forwarded to the downstream operators.
>>>
>>> Just wanted to lay the ground for discussions so we are all on the same
>>> page before chatting more.
>>>
>>>
>>> Guozhang
>>>
>>
>>
>> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>>
>>> Hi Matthias,
>>>
>>> Thanks a lot for correcting. It is a leftover from the past designs when
>>> punctuate() was not deprecated.
>>> I corrected.
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>> I just re-read the KIP.
>>>>
>>>> One minor comment: we don't need to introduce any deprecated methods.
>>>> Thus, RichValueTransformer#punctuate can be removed completely instead
>>>> of introducing it as deprecated.
>>>>
>>>> Otherwise looks good to me.
>>>>
>>>> Thanks for being so patient!
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>>
>>>>> Jeyhun,
>>>>>
>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
>>>>> carefully if we should add this functionality to the DSL layer moving
>>>>> forward since from what we discovered working on it the conclusion is
>>>>>
>>>> that
>>>>
>>>>> it would require revamping the public APIs quite a lot, and it's not
>>>>>
>>>> clear
>>>>
>>>>> if it is a good trade-off than asking users to call process() instead.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com>
>>>>> wrote:
>>>>>
>>>>> Hi Jeyhun, thanks, looks good.
>>>>>> Do we need to remove the line that says:
>>>>>>
>>>>>>     - on-demand commit() feature
>>>>>>
>>>>>> Cheers,
>>>>>> Damian
>>>>>>
>>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>>>>>>
>>>>> wrote:
>>>>
>>>>> Hi,
>>>>>>>
>>>>>>> I removed the 'commit()' feature, as we discussed. It simplified  the
>>>>>>> overall design of KIP a lot.
>>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Jeyhun
>>>>>>>
>>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <
>>>>>>> matthias@confluent.io
>>>>>>> wrote:
>>>>>>>
>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>>
>>>>>>>> but also we need a commit() method
>>>>>>>>>
>>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>>
>>>>>>>>> Hi Matthias,
>>>>>>>>>
>>>>>>>>> Thanks for your comments. I agree that this is not the best way to
>>>>>>>>>
>>>>>>>> do.
>>>>>>
>>>>>>> A
>>>>>>>
>>>>>>>> bit of history behind this design.
>>>>>>>>>
>>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as an
>>>>>>>>>
>>>>>>>> argument
>>>>>>>>
>>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>>>>>
>>>>>>>> flexibility
>>>>>>>
>>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>>>> information and not Record level info. The only thing we need ij
>>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>>
>>>>>>>>> So, as far as I understood, we need recor context (offset,
>>>>>>>>> timestamp
>>>>>>>>>
>>>>>>>> and
>>>>>>>
>>>>>>>> etc) but also we need a commit() method ( we dont want to provide
>>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>>>>
>>>>>>>> ProcessorContext.commit()
>>>>>>>>
>>>>>>>>> ).
>>>>>>>>>
>>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>>>>>>
>>>>>>>> RecordContext
>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>>> ProcessorContext() .
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>>>>>>
>>>>>>>> KIP/included
>>>>>>
>>>>>>> jiras please let me know.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Jeyhun
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <
>>>>>>>>> matthias@confluent.io
>>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> I am personally still not convinced, that we should add `commit()`
>>>>>>>>>>
>>>>>>>>> at
>>>>>>
>>>>>>> all.
>>>>>>>>
>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a
>>>>>>>>>> little
>>>>>>>>>> bit? Isn't requesting commits a low level API that should not be
>>>>>>>>>>
>>>>>>>>> exposed
>>>>>>>
>>>>>>>> in the DSL? Just want to understand the motivation better. Why would
>>>>>>>>>> anybody that uses the DSL ever want to request a commit? To me,
>>>>>>>>>> requesting commits is useful if you manipulated state explicitly,
>>>>>>>>>>
>>>>>>>>> ie,
>>>>>>
>>>>>>> via Processor API.
>>>>>>>>>>
>>>>>>>>>> Also, for the solution: it seem rather unnatural to me, that we
>>>>>>>>>> add
>>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>>>>>>
>>>>>>>>> `RecordContext`
>>>>>>>
>>>>>>>> is an helper object that provide access to record meta data.
>>>>>>>>>>
>>>>>>>>> Requesting
>>>>>>>
>>>>>>>> a commit is something quite different. Additionally, a commit does
>>>>>>>>>>
>>>>>>>>> not
>>>>>>
>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>>>>>>>
>>>>>>>>> record.
>>>>>>>
>>>>>>>> To me, this does not seem to be a sound API design if we follow this
>>>>>>>>>>
>>>>>>>>> path.
>>>>>>>>
>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>>
>>>>>>>>>>> I have some comments, to make sure that there is no
>>>>>>>>>>>
>>>>>>>>>> misunderstanding.
>>>>>>
>>>>>>>
>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>>>>>>>
>>>>>>>>>> enforce
>>>>>>>>
>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>
>>>>>>>>>>> implementation
>>>>>>>>
>>>>>>>>> of
>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>>>>>>>>
>>>>>>>>>>> changed
>>>>>>>
>>>>>>>> to
>>>>>>>>
>>>>>>>>> this call.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`.
>>>>>>>>>>> The
>>>>>>>>>>>
>>>>>>>>>> main
>>>>>>>
>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>>>>>>>>>> `RecordContext` is the one which is provided in Rich interfaces.
>>>>>>>>>>> So
>>>>>>>>>>>
>>>>>>>>>> if
>>>>>>>
>>>>>>>> user
>>>>>>>>>>
>>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>>>>>>
>>>>>>>>>> `RecordContext`
>>>>>>>>
>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>>>>>>>>
>>>>>>>>>> KIP-159):
>>>>>>
>>>>>>> @Override
>>>>>>>>>>>      public void process(final K1 key, final V1 value) {
>>>>>>>>>>>
>>>>>>>>>>>          recordContext = new RecordContext() {               //
>>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>>              @Override
>>>>>>>>>>>              public void commit() {
>>>>>>>>>>>                  context().commit();
>>>>>>>>>>>              }
>>>>>>>>>>>
>>>>>>>>>>>              @Override
>>>>>>>>>>>              public long offset() {
>>>>>>>>>>>                  return context().recordContext().offset();
>>>>>>>>>>>              }
>>>>>>>>>>>
>>>>>>>>>>>              @Override
>>>>>>>>>>>              public long timestamp() {
>>>>>>>>>>>                  return context().recordContext().timestamp();
>>>>>>>>>>>              }
>>>>>>>>>>>
>>>>>>>>>>>              @Override
>>>>>>>>>>>              public String topic() {
>>>>>>>>>>>                  return context().recordContext().topic();
>>>>>>>>>>>              }
>>>>>>>>>>>
>>>>>>>>>>>              @Override
>>>>>>>>>>>              public int partition() {
>>>>>>>>>>>                  return context().recordContext().partition();
>>>>>>>>>>>              }
>>>>>>>>>>>        };
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case
>>>>>>>>>>>
>>>>>>>>>> IMO.
>>>>>>
>>>>>>>
>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>
>>>>>>>>>> `ProcessorRecordContext`,
>>>>>>>>
>>>>>>>>> so
>>>>>>>>>>
>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()`
>>>>>>>>>>> just
>>>>>>>>>>>
>>>>>>>>>> only
>>>>>>>>
>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
>>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite amount
>>>>>>>>>>>
>>>>>>>>>> of
>>>>>>
>>>>>>> methods and it is logical to enable inheritance between those two.
>>>>>>>>>>>
>>>>>>>>>>> 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
>>>>>>>>>>>>
>>>>>>>>>>> snippet
>>>>>>>
>>>>>>>> below 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 has gone though many
>>>>>>>>>>>
>>>>>>>>>> changes
>>>>>>>>
>>>>>>>>> based on user/developer needs, both in
>>>>>>>>>>>
>>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <
>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>>>>>>>>
>>>>>>>>>>> KAFKA-3907
>>>>>>>
>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>>> this 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:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>>>>>>>>
>>>>>>>>>>> enforce
>>>>>>>>
>>>>>>>>> user to consolidate this call as
>>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>>>>>>>
>>>>>>>>>>> implementation
>>>>>>>>
>>>>>>>>> of
>>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>>>>>>>>
>>>>>>>>>>> changed
>>>>>>>
>>>>>>>> to
>>>>>>>>
>>>>>>>>> this call.
>>>>>>>>>>>>
>>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>>>>>
>>>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>
>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>>
>>>>>>>>>>>> 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
>>>>>>>>>>>>
>>>>>>>>>>> snippet
>>>>>>>
>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>>>>>>>
>>>>>>>>>>> matthias@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.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -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
>>>>>>>
>>>>>>>> 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/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. Sax <
>>>>>>>>>>>>>>
>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 DSL users should not worry about
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>
>>>>>>> and
>>>>>>>>
>>>>>>>>> if
>>>>>>>>>>>>
>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> dummy
>>>>>>
>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> argument
>>>>>>>
>>>>>>>> to
>>>>>>>>
>>>>>>>>> Rich
>>>>>>>>>>>>>
>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>      VR apply(final V value,
>>>>>>>>>>>>>>>>               final K key,
>>>>>>>>>>>>>>>>               final RecordContext recordContext);
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>>      VR apply(final V value,
>>>>>>>>>>>>>>>>               final K key,
>>>>>>>>>>>>>>>>               final RecordContext recordContext,
>>>>>>>>>>>>>>>>               final ProcessorContext processorContext);
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>
>>>>>>>> and
>>>>>>>>>>>>
>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only that
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> single
>>>>>>>>
>>>>>>>>> record
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K
>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> final
>>>>>>>>
>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> their
>>>>>>
>>>>>>> pairing
>>>>>>>>>>>>
>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>
>>>>>>> super
>>>>>>>>>>>>
>>>>>>>>>>>>> K,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>>                                     final Serde<KR>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>
>>>>>>>                                     final Serde<V>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>
>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> super
>>>>>>>
>>>>>>>> K,
>>>>>>>>
>>>>>>>>> ?
>>>>>>>>>>>>
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>>                                   final Serde<K>
>>>>>>>>>>>>>>>>>>> keySerde,
>>>>>>>>>>>>>>>>>>>                                   final Serde<V>
>>>>>>>>>>>>>>>>>>> valSerde);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> combo
>>>>>>>>
>>>>>>>>> of
>>>>>>>>>>>>
>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> adder /
>>>>>>
>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> functions
>>>>>>>
>>>>>>>> for
>>>>>>>>>>>>
>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> want
>>>>>>>
>>>>>>>> to
>>>>>>>>
>>>>>>>>> access
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> example:
>>>>>>>>>>
>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>                                   final
>>>>>>>>>>>>>>>>>>> RichKeyValueMapper<?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> super
>>>>>>>>
>>>>>>>>> K, ?
>>>>>>>>>>>>>
>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>>   V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> super
>>>>>>>
>>>>>>>> K,
>>>>>>>>
>>>>>>>>> ?
>>>>>>>>>>>>
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> its
>>>>>>>
>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> o.a.k.processor.internals
>>>>>>>>
>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> maintenance
>>>>>>>
>>>>>>>> (both
>>>>>>>>>>>>>
>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> methods),  I
>>>>>>>>>>>>
>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext and
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> in
>>>>>>
>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K
>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> final
>>>>>>>>
>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8, users
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> that
>>>>>>
>>>>>>> would
>>>>>>>>>>>>
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> likely
>>>>>>>>
>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than
>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> the
>>>>>>>
>>>>>>>> dummy
>>>>>>>>>>>>
>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> like
>>>>>>
>>>>>>> to
>>>>>>>>
>>>>>>>>> make
>>>>>>>>>>>>>
>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>


-- 
-- Guozhang

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

Posted by Jan Filipiak <Ja...@trivago.com>.
a remark of mine that got missed during migration:

There is this problem that even though we have source.table.filter.join
the state-fullness happens at the table step not a the join step. In a 
filter
we still going to present change.oldValue to the filter even though the
record context() is for change.newValue. I would go as far as applying
the filter before the table processor. Not to just get KIP-159, but because
I think its a side effect of a non ideal topology layout. If i can 
filter 99% of my
records. my state could be way smaller. Also widely escalates the context
of the KIP

I can only see upsides of executing the filter first.

Best Jan



On 20.11.2017 22:22, Matthias J. Sax wrote:
> I am moving this back to the DISCUSS thread... Last 10 emails were sent
> to VOTE thread.
>
> Copying Guozhang's last summary below. Thanks for this summary. Very
> comprehensive!
>
> It seems, we all agree, that the current implementation of the context
> at PAPI level is ok, but we should not leak it into DSL.
>
> Thus, we can go with (2) or (3), were (3) is an extension to (2)
> carrying the context to more operators than just sources. It also seems,
> that we all agree, that many-to-one operations void the context.
>
> I still think, that just going with plain (2) is too restrictive -- but
> I am also fine if we don't go with the full proposal of (3).
>
> Also note, that the two operators filter() and filterNot() don't modify
> the record and thus for both, it would be absolutely valid to keep the
> context.
>
> I personally would keep the context for at least all one-to-one
> operators. One-to-many is debatable and I am fine to not carry the
> context further: at least the offset information is questionable for
> this case -- note thought, that semantically, the timestamp is inherited
> via one-to-many, and I also think this applies to "topic" and
> "partition". Thus, I think it's still valuable information we can carry
> downstreams.
>
>
> -Matthias
>
>> Jan: which approach are you referring to as "the approach that is on the
>> table would be perfect"?
>>
>> Note that in today's PAPI layer we are already effectively exposing the
>> record context which has the issues that we have been discussing right now,
>> and its semantics is always referring to the "processing record" at hand.
>> More specifically, we can think of processing a record a bit different:
>>
>> 1) the record traversed the topology from source to sink, it may be
>> transformed into new object or even generate multiple new objects (think:
>> branch) along the traversal. And the record context is referring to this
>> processing record. Here the "lifetime" of the record lasts for the entire
>> topology traversal and any new records of this traversal is treated as
>> different transformed values of this record (this applies to join and
>> aggregations as well).
>>
>> 2) the record being processed is wiped out in the first operator after the
>> source, and NEW records are forwarded to downstream operators. I.e. each
>> record only lives between two adjacent operators, once it reached the new
>> operator it's lifetime has ended and new records are generated.
>>
>> I think in the past we have talked about Streams under both context, and we
>> do not have a clear agreement. I agree that 2) is logically more
>> understandable for users as it does not leak any internal implementation
>> details (e.g. for stream-table joins, table record's traversal ends at the
>> join operator as it is only be materialized, while stream record's
>> traversal goes through the join operator to further down until sinks).
>> However if we are going to interpret following 2) above then even for
>> non-stateful operators we would not inherit record context. What we're
>> discussing now, seems to infer a third semantics:
>>
>> 3) a record would traverse "through" one-to-one (non-stateful) operators,
>> will "replicate" at one-to-many (non-stateful) operators (think: "mapValues"
>>   ) and will "end" at many-to-one (stateful) operators where NEW records
>> will be generated and forwarded to the downstream operators.
>>
>> Just wanted to lay the ground for discussions so we are all on the same
>> page before chatting more.
>>
>>
>> Guozhang
>
>
> On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
>> Hi Matthias,
>>
>> Thanks a lot for correcting. It is a leftover from the past designs when
>> punctuate() was not deprecated.
>> I corrected.
>>
>> Cheers,
>> Jeyhun
>>
>> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>>> I just re-read the KIP.
>>>
>>> One minor comment: we don't need to introduce any deprecated methods.
>>> Thus, RichValueTransformer#punctuate can be removed completely instead
>>> of introducing it as deprecated.
>>>
>>> Otherwise looks good to me.
>>>
>>> Thanks for being so patient!
>>>
>>>
>>> -Matthias
>>>
>>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>>> Jeyhun,
>>>>
>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
>>>> carefully if we should add this functionality to the DSL layer moving
>>>> forward since from what we discovered working on it the conclusion is
>>> that
>>>> it would require revamping the public APIs quite a lot, and it's not
>>> clear
>>>> if it is a good trade-off than asking users to call process() instead.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com> wrote:
>>>>
>>>>> Hi Jeyhun, thanks, looks good.
>>>>> Do we need to remove the line that says:
>>>>>
>>>>>     - on-demand commit() feature
>>>>>
>>>>> Cheers,
>>>>> Damian
>>>>>
>>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>>> wrote:
>>>>>> Hi,
>>>>>>
>>>>>> I removed the 'commit()' feature, as we discussed. It simplified  the
>>>>>> overall design of KIP a lot.
>>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <matthias@confluent.io
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>>
>>>>>>>> but also we need a commit() method
>>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>>> Hi Matthias,
>>>>>>>>
>>>>>>>> Thanks for your comments. I agree that this is not the best way to
>>>>> do.
>>>>>> A
>>>>>>>> bit of history behind this design.
>>>>>>>>
>>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as an
>>>>>>> argument
>>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>>> flexibility
>>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>>> information and not Record level info. The only thing we need ij
>>>>>>>> ProcessorContext is commit() method.
>>>>>>>>
>>>>>>>> So, as far as I understood, we need recor context (offset, timestamp
>>>>>> and
>>>>>>>> etc) but also we need a commit() method ( we dont want to provide
>>>>>>>> ProcessorContext as a parameter so users can use
>>>>>>> ProcessorContext.commit()
>>>>>>>> ).
>>>>>>>>
>>>>>>>> As a result, I thought to “propagate” commit() call from
>>>>> RecordContext
>>>>>> to
>>>>>>>> ProcessorContext() .
>>>>>>>>
>>>>>>>>
>>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>>> KIP/included
>>>>>>>> jiras please let me know.
>>>>>>>>
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <matthias@confluent.io
>>>>>>> wrote:
>>>>>>>>> I am personally still not convinced, that we should add `commit()`
>>>>> at
>>>>>>> all.
>>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a little
>>>>>>>>> bit? Isn't requesting commits a low level API that should not be
>>>>>> exposed
>>>>>>>>> in the DSL? Just want to understand the motivation better. Why would
>>>>>>>>> anybody that uses the DSL ever want to request a commit? To me,
>>>>>>>>> requesting commits is useful if you manipulated state explicitly,
>>>>> ie,
>>>>>>>>> via Processor API.
>>>>>>>>>
>>>>>>>>> Also, for the solution: it seem rather unnatural to me, that we add
>>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>>> `RecordContext`
>>>>>>>>> is an helper object that provide access to record meta data.
>>>>>> Requesting
>>>>>>>>> a commit is something quite different. Additionally, a commit does
>>>>> not
>>>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>>> record.
>>>>>>>>> To me, this does not seem to be a sound API design if we follow this
>>>>>>> path.
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>>> Hi,
>>>>>>>>>>
>>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>>
>>>>>>>>>> I have some comments, to make sure that there is no
>>>>> misunderstanding.
>>>>>>>>>>
>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>>> enforce
>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>> implementation
>>>>>>>>>>> of
>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>> changed
>>>>>>> to
>>>>>>>>>>> this call.
>>>>>>>>>>
>>>>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`. The
>>>>>> main
>>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>>>>>>>>> `RecordContext` is the one which is provided in Rich interfaces. So
>>>>>> if
>>>>>>>>> user
>>>>>>>>>> wants to commit, then there should be some method inside
>>>>>>> `RecordContext`
>>>>>>>>> to
>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>>> KIP-159):
>>>>>>>>>> @Override
>>>>>>>>>>      public void process(final K1 key, final V1 value) {
>>>>>>>>>>
>>>>>>>>>>          recordContext = new RecordContext() {               //
>>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>>              @Override
>>>>>>>>>>              public void commit() {
>>>>>>>>>>                  context().commit();
>>>>>>>>>>              }
>>>>>>>>>>
>>>>>>>>>>              @Override
>>>>>>>>>>              public long offset() {
>>>>>>>>>>                  return context().recordContext().offset();
>>>>>>>>>>              }
>>>>>>>>>>
>>>>>>>>>>              @Override
>>>>>>>>>>              public long timestamp() {
>>>>>>>>>>                  return context().recordContext().timestamp();
>>>>>>>>>>              }
>>>>>>>>>>
>>>>>>>>>>              @Override
>>>>>>>>>>              public String topic() {
>>>>>>>>>>                  return context().recordContext().topic();
>>>>>>>>>>              }
>>>>>>>>>>
>>>>>>>>>>              @Override
>>>>>>>>>>              public int partition() {
>>>>>>>>>>                  return context().recordContext().partition();
>>>>>>>>>>              }
>>>>>>>>>>        };
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case
>>>>> IMO.
>>>>>>>>>>
>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>> `ProcessorRecordContext`,
>>>>>>>>> so
>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>
>>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()` just
>>>>>>> only
>>>>>>>>> to
>>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
>>>>>>>>>> `ProcessorRecordContext`, just because of they share quite amount
>>>>> of
>>>>>>>>>> methods and it is logical to enable inheritance between those two.
>>>>>>>>>>
>>>>>>>>>> 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
>>>>>> snippet
>>>>>>>>>>> below 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 has gone though many
>>>>>>> changes
>>>>>>>>>> based on user/developer needs, both in
>>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>>> KAFKA-3907
>>>>>>>>> with
>>>>>>>>>>> this 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:
>>>>>>>>>>>
>>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>>> enforce
>>>>>>>>>>> user to consolidate this call as
>>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>>> implementation
>>>>>>>>>>> of
>>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>>> changed
>>>>>>> to
>>>>>>>>>>> this call.
>>>>>>>>>>>
>>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>>
>>>>>>>>>>> 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
>>>>>> snippet
>>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>>> matthias@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.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -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
>>>>>>>>>>> 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/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. Sax <
>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 DSL users should not worry about
>>>>> --
>>>>>>> and
>>>>>>>>>>> if
>>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>>> dummy
>>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>> 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
>>>>>> argument
>>>>>>> to
>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>      VR apply(final V value,
>>>>>>>>>>>>>>>               final K key,
>>>>>>>>>>>>>>>               final RecordContext recordContext);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>>      VR apply(final V value,
>>>>>>>>>>>>>>>               final K key,
>>>>>>>>>>>>>>>               final RecordContext recordContext,
>>>>>>>>>>>>>>>               final ProcessorContext processorContext);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>>> ProcessorContext
>>>>>>>>>>> and
>>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>>> wangguoz@gmail.com
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only that
>>>>>>> single
>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K key,
>>>>>>> final
>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>>> their
>>>>>>>>>>> pairing
>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>> RichKeyValueMapper<?
>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>                                     final Serde<KR>
>>>>> keySerde,
>>>>>>>>>>>>>>>>>>                                     final Serde<V>
>>>>> valSerde);
>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>> super
>>>>>>> K,
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>                                   final Serde<K> keySerde,
>>>>>>>>>>>>>>>>>>                                   final Serde<V> valSerde);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>>>>> combo
>>>>>>>>>>> of
>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>> adder /
>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>> functions
>>>>>>>>>>> for
>>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>>>>> want
>>>>>>> to
>>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>>>>> example:
>>>>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>>                                   final RichKeyValueMapper<?
>>>>>>> super
>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>   V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>> super
>>>>>>> K,
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
>>>>>> its
>>>>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>> o.a.k.processor.internals
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>>> maintenance
>>>>>>>>>>>> (both
>>>>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>>> methods),  I
>>>>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext and
>>>>> in
>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>>      VR apply(final V1 value1, final V2 value2, final K key,
>>>>>>> final
>>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8, users
>>>>> that
>>>>>>>>>>> would
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
>>>>>>> likely
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
>>>>>> the
>>>>>>>>>>> dummy
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
>>>>> like
>>>>>>> to
>>>>>>>>>>>> make
>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>>> their
>>>>>>>>>>>> pairing
>>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>>> RichKeyValueMapper<?
>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>>                                     final Serde<KR>
>>>>> keySerde,
>>>>>>>>>>>>>>>>>>                                     final Serde<V>
>>>>> valSerde);
>>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>> super
>>>>>>> K,
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>>                                   final Serde<K> keySerde,
>>>>>>>>>>>>>>>>>>                                   final Serde<V> valSerde);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>>>>> combo
>>>>>>>>>>> of
>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>>> adder /
>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>>> functions
>>>>>>>>>>> for
>>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>>>>> want
>>>>>>> to
>>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>>>>> example:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>>                                   final RichKeyValueMapper<?
>>>>>>> super
>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>>   V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>>                                   final RichValueJoiner<?
>>>>>> super
>>>>>>> K,
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>                               final RichAggregator<? super
>>>>> K,
>>>>>> ?
>>>>>>>>>>> super
>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>> aggregator,
>>>>>>>>>>>>>>>>>>                               final Materialized<K, VR,
>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
>>>>>>>>> deprecated
>>>>>>>>>>>> so
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>> not need to add its rich functions any more.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
>>>>> make
>>>>>>> its
>>>>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>>>>> RichInitializer<VR,
>>>>>>>>>>> K>
>>>>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>                                         final
>>>>> RichAggregator<?
>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>> super V, VR> aggregator);
>>>>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>>>>> RichInitializer<VR,
>>>>>>>>>>> K>
>>>>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>                                         final
>>>>> RichAggregator<?
>>>>>>>>>>> super
>>>>>>>>>>>> K,
>>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>> super V, VR> aggregator,
>>>>>>>>>>>>>>>>>>                                         final
>>>>> Materialized<K,
>>>>>>> VR,
>>>>>>>>>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>>> o.a.k.processor.internals
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> RecordContext? Conceptually I think it would better staying
>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> ProcessorContext. Do you find this not doable in the
>>>>> internal
>>>>>>>>>>>>>>>>>> implementations?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <
>>>>> yuzhihong@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>     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 <
>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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 to KIP.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>>                               final Aggregator<? super
>>>>> K, ?
>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> related
>>>>>>>>>>>>>>>>>>>> method. So, I had to overload all possible their Rich
>>>>>>>>>>>> counterparts:
>>>>>>>>>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>                               final Aggregator<? super K,
>>>>> ?
>>>>>>>>> super
>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>                               final RichAggregator<? super
>>>>>> K,
>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>>                               final Materialized<K, VR,
>>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>                               final RichAggregator<? super
>>>>>> K,
>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>                               final Aggregator<? super K,
>>>>> ?
>>>>>>>>> super
>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>>                               final Materialized<K, VR,
>>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> // both adder and subtractor are rich
>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>                               final RichAggregator<? super
>>>>>> K,
>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>                               final RichAggregator<? super
>>>>>> K,
>>>>>>> ?
>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>>                               final Materialized<K, VR,
>>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>>>>     void commit () {
>>>>>>>>>>>>>>>>>>>>>       throw new UnsupportedOperationException("commit()
>>>>> is
>>>>>>> not
>>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>>>>> the
>>>>>>> PR
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - I added some comments both inside and outside the code
>>>>>>>>>>> snippets
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>> Specifically, for the code snippet above, we add
>>>>> *commit()*
>>>>>>>>>>> method
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> *RecordContext* interface.
>>>>>>>>>>>>>>>>>>>> However, we want  *commit()* method to be used only for
>>>>>>>>>>>>>>>>> *RecordContext*
>>>>>>>>>>>>>>>>>>>> instances (at least for now), so we add
>>>>>>>>>>>>>>>> UnsupportedOperationException
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> all classes/interfaces that extend/implement
>>>>>> *RecordContext.*
>>>>>>>>>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
>>>>>>> within
>>>>>>>>>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
>>>>>> Rich
>>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>>> inside Processors.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
>>>>>> yuzhihong@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>>                               final Aggregator<? super
>>>>> K, ?
>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>    public RecordContext recordContext() {
>>>>>>>>>>>>>>>>>>>>>      return this.recordContext();
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>     void commit () {
>>>>>>>>>>>>>>>>>>>>>       throw new UnsupportedOperationException("commit()
>>>>> is
>>>>>>> not
>>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>>>>> the
>>>>>>> PR
>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>> Cheers
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>>>> 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 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
>>>>>> KIP-159
>>>>>>> is
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> voted
>>>>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as
>>>>> an
>>>>>>>>>>>>>>>>> "atomic"
>>>>>>>>>>>>>>>>>>>>> merge).
>>>>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there
>>>>> are
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please
>>>>> correct
>>>>>> me
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>>>>> or I
>>>>>>>>>>>>>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and
>>>>> vote
>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>>>>>>>>>>>>>> yuzhihong@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun
>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>>>>>>>>>>>>>> RichXXX
>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> revert
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>>>>>>>>>>>>>> RichXXXX
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>>>>>>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>>>>>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Matthias, Damian, all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your comments and sorry for
>>>>>>>>>>>>>>>>>>> super-late
>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sure, the DSL refactoring is not blocking
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> KIP.



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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I am moving this back to the DISCUSS thread... Last 10 emails were sent
to VOTE thread.

Copying Guozhang's last summary below. Thanks for this summary. Very
comprehensive!

It seems, we all agree, that the current implementation of the context
at PAPI level is ok, but we should not leak it into DSL.

Thus, we can go with (2) or (3), were (3) is an extension to (2)
carrying the context to more operators than just sources. It also seems,
that we all agree, that many-to-one operations void the context.

I still think, that just going with plain (2) is too restrictive -- but
I am also fine if we don't go with the full proposal of (3).

Also note, that the two operators filter() and filterNot() don't modify
the record and thus for both, it would be absolutely valid to keep the
context.

I personally would keep the context for at least all one-to-one
operators. One-to-many is debatable and I am fine to not carry the
context further: at least the offset information is questionable for
this case -- note thought, that semantically, the timestamp is inherited
via one-to-many, and I also think this applies to "topic" and
"partition". Thus, I think it's still valuable information we can carry
downstreams.


-Matthias

> Jan: which approach are you referring to as "the approach that is on the
> table would be perfect"?
> 
> Note that in today's PAPI layer we are already effectively exposing the
> record context which has the issues that we have been discussing right now,
> and its semantics is always referring to the "processing record" at hand.
> More specifically, we can think of processing a record a bit different:
> 
> 1) the record traversed the topology from source to sink, it may be
> transformed into new object or even generate multiple new objects (think:
> branch) along the traversal. And the record context is referring to this
> processing record. Here the "lifetime" of the record lasts for the entire
> topology traversal and any new records of this traversal is treated as
> different transformed values of this record (this applies to join and
> aggregations as well).
> 
> 2) the record being processed is wiped out in the first operator after the
> source, and NEW records are forwarded to downstream operators. I.e. each
> record only lives between two adjacent operators, once it reached the new
> operator it's lifetime has ended and new records are generated.
> 
> I think in the past we have talked about Streams under both context, and we
> do not have a clear agreement. I agree that 2) is logically more
> understandable for users as it does not leak any internal implementation
> details (e.g. for stream-table joins, table record's traversal ends at the
> join operator as it is only be materialized, while stream record's
> traversal goes through the join operator to further down until sinks).
> However if we are going to interpret following 2) above then even for
> non-stateful operators we would not inherit record context. What we're
> discussing now, seems to infer a third semantics:
> 
> 3) a record would traverse "through" one-to-one (non-stateful) operators,
> will "replicate" at one-to-many (non-stateful) operators (think: "mapValues"
>  ) and will "end" at many-to-one (stateful) operators where NEW records
> will be generated and forwarded to the downstream operators.
> 
> Just wanted to lay the ground for discussions so we are all on the same
> page before chatting more.
> 
> 
> Guozhang



On 11/6/17 1:41 PM, Jeyhun Karimov wrote:
> Hi Matthias,
> 
> Thanks a lot for correcting. It is a leftover from the past designs when
> punctuate() was not deprecated.
> I corrected.
> 
> Cheers,
> Jeyhun
> 
> On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> I just re-read the KIP.
>>
>> One minor comment: we don't need to introduce any deprecated methods.
>> Thus, RichValueTransformer#punctuate can be removed completely instead
>> of introducing it as deprecated.
>>
>> Otherwise looks good to me.
>>
>> Thanks for being so patient!
>>
>>
>> -Matthias
>>
>> On 11/1/17 9:16 PM, Guozhang Wang wrote:
>>> Jeyhun,
>>>
>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
>>> carefully if we should add this functionality to the DSL layer moving
>>> forward since from what we discovered working on it the conclusion is
>> that
>>> it would require revamping the public APIs quite a lot, and it's not
>> clear
>>> if it is a good trade-off than asking users to call process() instead.
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com> wrote:
>>>
>>>> Hi Jeyhun, thanks, looks good.
>>>> Do we need to remove the line that says:
>>>>
>>>>    - on-demand commit() feature
>>>>
>>>> Cheers,
>>>> Damian
>>>>
>>>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
>> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> I removed the 'commit()' feature, as we discussed. It simplified  the
>>>>> overall design of KIP a lot.
>>>>> If it is ok, I would like to start a VOTE thread.
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <matthias@confluent.io
>>>
>>>>> wrote:
>>>>>
>>>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>>>
>>>>>>> but also we need a commit() method
>>>>>>
>>>>>> I would just not provide `commit()` at DSL level and close the
>>>>>> corresponding Jira as "not a problem" or similar.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>>>> Hi Matthias,
>>>>>>>
>>>>>>> Thanks for your comments. I agree that this is not the best way to
>>>> do.
>>>>> A
>>>>>>> bit of history behind this design.
>>>>>>>
>>>>>>> Prior doing this, I tried to provide ProcessorContext itself as an
>>>>>> argument
>>>>>>> in Rich interfaces. However, we dont want to give users that
>>>>> flexibility
>>>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>>>> information and not Record level info. The only thing we need ij
>>>>>>> ProcessorContext is commit() method.
>>>>>>>
>>>>>>> So, as far as I understood, we need recor context (offset, timestamp
>>>>> and
>>>>>>> etc) but also we need a commit() method ( we dont want to provide
>>>>>>> ProcessorContext as a parameter so users can use
>>>>>> ProcessorContext.commit()
>>>>>>> ).
>>>>>>>
>>>>>>> As a result, I thought to “propagate” commit() call from
>>>> RecordContext
>>>>> to
>>>>>>> ProcessorContext() .
>>>>>>>
>>>>>>>
>>>>>>> If there is a misunderstanding in motvation/discussion of
>>>> KIP/included
>>>>>>> jiras please let me know.
>>>>>>>
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Jeyhun
>>>>>>>
>>>>>>>
>>>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <matthias@confluent.io
>>>>>
>>>>>> wrote:
>>>>>>>
>>>>>>>> I am personally still not convinced, that we should add `commit()`
>>>> at
>>>>>> all.
>>>>>>>>
>>>>>>>> @Guozhang: you created the original Jira. Can you elaborate a little
>>>>>>>> bit? Isn't requesting commits a low level API that should not be
>>>>> exposed
>>>>>>>> in the DSL? Just want to understand the motivation better. Why would
>>>>>>>> anybody that uses the DSL ever want to request a commit? To me,
>>>>>>>> requesting commits is useful if you manipulated state explicitly,
>>>> ie,
>>>>>>>> via Processor API.
>>>>>>>>
>>>>>>>> Also, for the solution: it seem rather unnatural to me, that we add
>>>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>>>> `RecordContext`
>>>>>>>> is an helper object that provide access to record meta data.
>>>>> Requesting
>>>>>>>> a commit is something quite different. Additionally, a commit does
>>>> not
>>>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>>>> record.
>>>>>>>>
>>>>>>>> To me, this does not seem to be a sound API design if we follow this
>>>>>> path.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> Thanks for your suggestions.
>>>>>>>>>
>>>>>>>>> I have some comments, to make sure that there is no
>>>> misunderstanding.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>> enforce
>>>>>>>>>> user to consolidate this call as
>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>> implementation
>>>>>>>>>> of
>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>> changed
>>>>>> to
>>>>>>>>>> this call.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`. The
>>>>> main
>>>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>>>>>>>> `RecordContext` is the one which is provided in Rich interfaces. So
>>>>> if
>>>>>>>> user
>>>>>>>>> wants to commit, then there should be some method inside
>>>>>> `RecordContext`
>>>>>>>> to
>>>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>>>> KIP-159):
>>>>>>>>>
>>>>>>>>> @Override
>>>>>>>>>     public void process(final K1 key, final V1 value) {
>>>>>>>>>
>>>>>>>>>         recordContext = new RecordContext() {               //
>>>>>>>>> recordContext initialization is added in this KIP
>>>>>>>>>             @Override
>>>>>>>>>             public void commit() {
>>>>>>>>>                 context().commit();
>>>>>>>>>             }
>>>>>>>>>
>>>>>>>>>             @Override
>>>>>>>>>             public long offset() {
>>>>>>>>>                 return context().recordContext().offset();
>>>>>>>>>             }
>>>>>>>>>
>>>>>>>>>             @Override
>>>>>>>>>             public long timestamp() {
>>>>>>>>>                 return context().recordContext().timestamp();
>>>>>>>>>             }
>>>>>>>>>
>>>>>>>>>             @Override
>>>>>>>>>             public String topic() {
>>>>>>>>>                 return context().recordContext().topic();
>>>>>>>>>             }
>>>>>>>>>
>>>>>>>>>             @Override
>>>>>>>>>             public int partition() {
>>>>>>>>>                 return context().recordContext().partition();
>>>>>>>>>             }
>>>>>>>>>       };
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case
>>>> IMO.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>> `ProcessorRecordContext`,
>>>>>>>> so
>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()` just
>>>>>> only
>>>>>>>> to
>>>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
>>>>>>>>> `ProcessorRecordContext`, just because of they share quite amount
>>>> of
>>>>>>>>> methods and it is logical to enable inheritance between those two.
>>>>>>>>>
>>>>>>>>> 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
>>>>> snippet
>>>>>>>>>> below 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 has gone though many
>>>>>> changes
>>>>>>>>> based on user/developer needs, both in
>>>>>>>> documentation-/implementation-wise.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Jeyhun
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>>>> KAFKA-3907
>>>>>>>> with
>>>>>>>>>> this 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:
>>>>>>>>>>
>>>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>>>> enforce
>>>>>>>>>> user to consolidate this call as
>>>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>>>> implementation
>>>>>>>>>> of
>>>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>>>> changed
>>>>>> to
>>>>>>>>>> this call.
>>>>>>>>>>
>>>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>>>> `ProcessorRecordContext`, so
>>>>>>>>>> that it can implement the commit call itself.
>>>>>>>>>>
>>>>>>>>>> 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
>>>>> snippet
>>>>>>>>>> below would need to be updated as well.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>>>> matthias@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.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -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
>>>>>>>>>> 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/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. Sax <
>>>>>>>> matthias@confluent.io
>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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 DSL users should not worry about
>>>> --
>>>>>> and
>>>>>>>>>> if
>>>>>>>>>>>>> one really needs this, an advanced user can still insert a
>>>> dummy
>>>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>> 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
>>>>> argument
>>>>>> to
>>>>>>>>>>> Rich
>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>     VR apply(final V value,
>>>>>>>>>>>>>>              final K key,
>>>>>>>>>>>>>>              final RecordContext recordContext);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>>>     VR apply(final V value,
>>>>>>>>>>>>>>              final K key,
>>>>>>>>>>>>>>              final RecordContext recordContext,
>>>>>>>>>>>>>>              final ProcessorContext processorContext);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> However, in this case, a user can get confused as
>>>>> ProcessorContext
>>>>>>>>>> and
>>>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>>>> wangguoz@gmail.com
>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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 whole task up to this processed record, not only that
>>>>>> single
>>>>>>>>>>>>> record
>>>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
>>>>>> final
>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>> their
>>>>>>>>>> pairing
>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>> RichKeyValueMapper<?
>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>                                    final Serde<KR>
>>>> keySerde,
>>>>>>>>>>>>>>>>>                                    final Serde<V>
>>>> valSerde);
>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>>>> super
>>>>>> K,
>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>>>> combo
>>>>>>>>>> of
>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>> adder /
>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>> functions
>>>>>>>>>> for
>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>>>> want
>>>>>> to
>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>>>> example:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
>>>>>> super
>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>>>> super
>>>>>> K,
>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
>>>>> its
>>>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>> o.a.k.processor.internals
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>>>> maintenance
>>>>>>>>>>> (both
>>>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>>>> methods),  I
>>>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext and
>>>> in
>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
>>>>>> final
>>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8, users
>>>> that
>>>>>>>>>> would
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
>>>>>> likely
>>>>>>>>>> to
>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
>>>>> the
>>>>>>>>>> dummy
>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
>>>> like
>>>>>> to
>>>>>>>>>>> make
>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>>>> their
>>>>>>>>>>> pairing
>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>>>> RichKeyValueMapper<?
>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>>>                                    final Serde<KR>
>>>> keySerde,
>>>>>>>>>>>>>>>>>                                    final Serde<V>
>>>> valSerde);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>>>> super
>>>>>> K,
>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>>>> combo
>>>>>>>>>> of
>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>>>> adder /
>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>>>> functions
>>>>>>>>>> for
>>>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>>>> want
>>>>>> to
>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>>>> example:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
>>>>>> super
>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>>>> super
>>>>>> K,
>>>>>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>>> K,
>>>>> ?
>>>>>>>>>> super
>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> aggregator,
>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
>>>>>>>> deprecated
>>>>>>>>>>> so
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>> not need to add its rich functions any more.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
>>>> make
>>>>>> its
>>>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>>>> RichInitializer<VR,
>>>>>>>>>> K>
>>>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>                                        final
>>>> RichAggregator<?
>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super V, VR> aggregator);
>>>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>>>> RichInitializer<VR,
>>>>>>>>>> K>
>>>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>                                        final
>>>> RichAggregator<?
>>>>>>>>>> super
>>>>>>>>>>> K,
>>>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>>>> super V, VR> aggregator,
>>>>>>>>>>>>>>>>>                                        final
>>>> Materialized<K,
>>>>>> VR,
>>>>>>>>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>>>> o.a.k.processor.internals
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> RecordContext? Conceptually I think it would better staying
>>>>> in
>>>>>>>> the
>>>>>>>>>>>>>>>>> ProcessorContext. Do you find this not doable in the
>>>> internal
>>>>>>>>>>>>>>>>> implementations?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <
>>>> yuzhihong@gmail.com
>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>    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 <
>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 to KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>                              final Aggregator<? super
>>>> K, ?
>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
>>>>> in
>>>>>>>> the
>>>>>>>>>>>>>>>>> related
>>>>>>>>>>>>>>>>>>> method. So, I had to overload all possible their Rich
>>>>>>>>>>> counterparts:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
>>>> ?
>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>>>> K,
>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>>>> K,
>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
>>>> ?
>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // both adder and subtractor are rich
>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>>>> K,
>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>>>> K,
>>>>>> ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
>>>> is
>>>>>> not
>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>>>> the
>>>>>> PR
>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - I added some comments both inside and outside the code
>>>>>>>>>> snippets
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>> Specifically, for the code snippet above, we add
>>>> *commit()*
>>>>>>>>>> method
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> *RecordContext* interface.
>>>>>>>>>>>>>>>>>>> However, we want  *commit()* method to be used only for
>>>>>>>>>>>>>>>> *RecordContext*
>>>>>>>>>>>>>>>>>>> instances (at least for now), so we add
>>>>>>>>>>>>>>> UnsupportedOperationException
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> all classes/interfaces that extend/implement
>>>>> *RecordContext.*
>>>>>>>>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
>>>>>> within
>>>>>>>>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
>>>>> Rich
>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>> inside Processors.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
>>>>> yuzhihong@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>>>>                              final Aggregator<? super
>>>> K, ?
>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>   public RecordContext recordContext() {
>>>>>>>>>>>>>>>>>>>>     return this.recordContext();
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
>>>> is
>>>>>> not
>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>>>> the
>>>>>> PR
>>>>>>>>>> ?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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 Jeyhun Karimov <
>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
>>>>> KIP-159
>>>>>> is
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> voted
>>>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as
>>>> an
>>>>>>>>>>>>>>>> "atomic"
>>>>>>>>>>>>>>>>>>>> merge).
>>>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there
>>>> are
>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please
>>>> correct
>>>>> me
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>>>> or I
>>>>>>>>>>>>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and
>>>> vote
>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>>>>>>>>>>>>> yuzhihong@gmail.com
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun
>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>>>>>>>>>>>>> RichXXX
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> revert
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>>>>>>>>>>>>> RichXXXX
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>>>>>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>>>>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Matthias, Damian, all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your comments and sorry for
>>>>>>>>>>>>>>>>>> super-late
>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sure, the DSL refactoring is not blocking
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>
> 


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

Posted by Jeyhun Karimov <je...@gmail.com>.
Hi Matthias,

Thanks a lot for correcting. It is a leftover from the past designs when
punctuate() was not deprecated.
I corrected.

Cheers,
Jeyhun

On Mon, Nov 6, 2017 at 5:30 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> I just re-read the KIP.
>
> One minor comment: we don't need to introduce any deprecated methods.
> Thus, RichValueTransformer#punctuate can be removed completely instead
> of introducing it as deprecated.
>
> Otherwise looks good to me.
>
> Thanks for being so patient!
>
>
> -Matthias
>
> On 11/1/17 9:16 PM, Guozhang Wang wrote:
> > Jeyhun,
> >
> > I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
> > carefully if we should add this functionality to the DSL layer moving
> > forward since from what we discovered working on it the conclusion is
> that
> > it would require revamping the public APIs quite a lot, and it's not
> clear
> > if it is a good trade-off than asking users to call process() instead.
> >
> >
> > Guozhang
> >
> >
> > On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com> wrote:
> >
> >> Hi Jeyhun, thanks, looks good.
> >> Do we need to remove the line that says:
> >>
> >>    - on-demand commit() feature
> >>
> >> Cheers,
> >> Damian
> >>
> >> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com>
> wrote:
> >>
> >>> Hi,
> >>>
> >>> I removed the 'commit()' feature, as we discussed. It simplified  the
> >>> overall design of KIP a lot.
> >>> If it is ok, I would like to start a VOTE thread.
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <matthias@confluent.io
> >
> >>> wrote:
> >>>
> >>>> Thanks. I understand what you are saying, but I don't agree that
> >>>>
> >>>>> but also we need a commit() method
> >>>>
> >>>> I would just not provide `commit()` at DSL level and close the
> >>>> corresponding Jira as "not a problem" or similar.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> >>>>> Hi Matthias,
> >>>>>
> >>>>> Thanks for your comments. I agree that this is not the best way to
> >> do.
> >>> A
> >>>>> bit of history behind this design.
> >>>>>
> >>>>> Prior doing this, I tried to provide ProcessorContext itself as an
> >>>> argument
> >>>>> in Rich interfaces. However, we dont want to give users that
> >>> flexibility
> >>>>> and “power”. Moreover, ProcessorContext contains processor level
> >>>>> information and not Record level info. The only thing we need ij
> >>>>> ProcessorContext is commit() method.
> >>>>>
> >>>>> So, as far as I understood, we need recor context (offset, timestamp
> >>> and
> >>>>> etc) but also we need a commit() method ( we dont want to provide
> >>>>> ProcessorContext as a parameter so users can use
> >>>> ProcessorContext.commit()
> >>>>> ).
> >>>>>
> >>>>> As a result, I thought to “propagate” commit() call from
> >> RecordContext
> >>> to
> >>>>> ProcessorContext() .
> >>>>>
> >>>>>
> >>>>> If there is a misunderstanding in motvation/discussion of
> >> KIP/included
> >>>>> jiras please let me know.
> >>>>>
> >>>>>
> >>>>> Cheers,
> >>>>> Jeyhun
> >>>>>
> >>>>>
> >>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <matthias@confluent.io
> >>>
> >>>> wrote:
> >>>>>
> >>>>>> I am personally still not convinced, that we should add `commit()`
> >> at
> >>>> all.
> >>>>>>
> >>>>>> @Guozhang: you created the original Jira. Can you elaborate a little
> >>>>>> bit? Isn't requesting commits a low level API that should not be
> >>> exposed
> >>>>>> in the DSL? Just want to understand the motivation better. Why would
> >>>>>> anybody that uses the DSL ever want to request a commit? To me,
> >>>>>> requesting commits is useful if you manipulated state explicitly,
> >> ie,
> >>>>>> via Processor API.
> >>>>>>
> >>>>>> Also, for the solution: it seem rather unnatural to me, that we add
> >>>>>> `commit()` to `RecordContext` -- from my understanding,
> >>> `RecordContext`
> >>>>>> is an helper object that provide access to record meta data.
> >>> Requesting
> >>>>>> a commit is something quite different. Additionally, a commit does
> >> not
> >>>>>> commit a specific record but a `RecrodContext` is for a specific
> >>> record.
> >>>>>>
> >>>>>> To me, this does not seem to be a sound API design if we follow this
> >>>> path.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> >>>>>>> Hi,
> >>>>>>>
> >>>>>>> Thanks for your suggestions.
> >>>>>>>
> >>>>>>> I have some comments, to make sure that there is no
> >> misunderstanding.
> >>>>>>>
> >>>>>>>
> >>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> >>>> enforce
> >>>>>>>> user to consolidate this call as
> >>>>>>>> "processorContext.recordContext().commit()". And internal
> >>>> implementation
> >>>>>>>> of
> >>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> >>> changed
> >>>> to
> >>>>>>>> this call.
> >>>>>>>
> >>>>>>>
> >>>>>>> - I think we should not deprecate `ProcessorContext.commit()`. The
> >>> main
> >>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
> >>>>>>> `RecordContext` is the one which is provided in Rich interfaces. So
> >>> if
> >>>>>> user
> >>>>>>> wants to commit, then there should be some method inside
> >>>> `RecordContext`
> >>>>>> to
> >>>>>>> do so. Internally, `RecordContext.commit()` calls
> >>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
> >> KIP-159):
> >>>>>>>
> >>>>>>> @Override
> >>>>>>>     public void process(final K1 key, final V1 value) {
> >>>>>>>
> >>>>>>>         recordContext = new RecordContext() {               //
> >>>>>>> recordContext initialization is added in this KIP
> >>>>>>>             @Override
> >>>>>>>             public void commit() {
> >>>>>>>                 context().commit();
> >>>>>>>             }
> >>>>>>>
> >>>>>>>             @Override
> >>>>>>>             public long offset() {
> >>>>>>>                 return context().recordContext().offset();
> >>>>>>>             }
> >>>>>>>
> >>>>>>>             @Override
> >>>>>>>             public long timestamp() {
> >>>>>>>                 return context().recordContext().timestamp();
> >>>>>>>             }
> >>>>>>>
> >>>>>>>             @Override
> >>>>>>>             public String topic() {
> >>>>>>>                 return context().recordContext().topic();
> >>>>>>>             }
> >>>>>>>
> >>>>>>>             @Override
> >>>>>>>             public int partition() {
> >>>>>>>                 return context().recordContext().partition();
> >>>>>>>             }
> >>>>>>>       };
> >>>>>>>
> >>>>>>>
> >>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case
> >> IMO.
> >>>>>>>
> >>>>>>>
> >>>>>>> 2. Add the `task` reference to the impl class,
> >>>> `ProcessorRecordContext`,
> >>>>>> so
> >>>>>>>> that it can implement the commit call itself.
> >>>>>>>
> >>>>>>>
> >>>>>>> - Actually, I don't think that we need `commit()` in
> >>>>>>> `ProcessorRecordContext`. 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 `RecordContext()` just
> >>>> only
> >>>>>> to
> >>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
> >>>>>>> `ProcessorRecordContext`, just because of they share quite amount
> >> of
> >>>>>>> methods and it is logical to enable inheritance between those two.
> >>>>>>>
> >>>>>>> 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
> >>> snippet
> >>>>>>>> below 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 has gone though many
> >>>> changes
> >>>>>>> based on user/developer needs, both in
> >>>>>> documentation-/implementation-wise.
> >>>>>>>
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Jeyhun
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Thanks for the information Jeyhun. I had also forgot about
> >>> KAFKA-3907
> >>>>>> with
> >>>>>>>> this 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:
> >>>>>>>>
> >>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> >>>> enforce
> >>>>>>>> user to consolidate this call as
> >>>>>>>> "processorContext.recordContext().commit()". And internal
> >>>> implementation
> >>>>>>>> of
> >>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> >>> changed
> >>>> to
> >>>>>>>> this call.
> >>>>>>>>
> >>>>>>>> 2. Add the `task` reference to the impl class,
> >>>>>> `ProcessorRecordContext`, so
> >>>>>>>> that it can implement the commit call itself.
> >>>>>>>>
> >>>>>>>> 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
> >>> snippet
> >>>>>>>> below would need to be updated as well.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> >>>> matthias@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.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -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
> >>>>>>>> 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/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. Sax <
> >>>>>> matthias@confluent.io
> >>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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 DSL users should not worry about
> >> --
> >>>> and
> >>>>>>>> if
> >>>>>>>>>>> one really needs this, an advanced user can still insert a
> >> dummy
> >>>>>>>>>>> `transform` to request a commit from there.
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>>>>>>>>>>> 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
> >>> argument
> >>>> to
> >>>>>>>>> Rich
> >>>>>>>>>>>> methods:
> >>>>>>>>>>>>
> >>>>>>>>>>>> instead of
> >>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>     VR apply(final V value,
> >>>>>>>>>>>>              final K key,
> >>>>>>>>>>>>              final RecordContext recordContext);
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>> we can adopt
> >>>>>>>>>>>>
> >>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>>>>>     VR apply(final V value,
> >>>>>>>>>>>>              final K key,
> >>>>>>>>>>>>              final RecordContext recordContext,
> >>>>>>>>>>>>              final ProcessorContext processorContext);
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> However, in this case, a user can get confused as
> >>> ProcessorContext
> >>>>>>>> and
> >>>>>>>>>>>> RecordContext share some methods with the same name.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Cheers,
> >>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
> >>> wangguoz@gmail.com
> >>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 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 whole task up to this processed record, not only that
> >>>> single
> >>>>>>>>>>> record
> >>>>>>>>>>>>> itself.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> >>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> >>>> final
> >>>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> - Fixed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
> >> their
> >>>>>>>> pairing
> >>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> >> RichKeyValueMapper<?
> >>>>>>>> super
> >>>>>>>>> K,
> >>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>>>>>                                    final Serde<KR>
> >> keySerde,
> >>>>>>>>>>>>>>>                                    final Serde<V>
> >> valSerde);
> >>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>>>>>>>                                  final RichValueJoiner<?
> >>> super
> >>>> K,
> >>>>>>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Fixed
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> >>>> combo
> >>>>>>>> of
> >>>>>>>>>>> both
> >>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
> >> adder /
> >>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> >>> functions
> >>>>>>>> for
> >>>>>>>>>>>>> both;
> >>>>>>>>>>>>>>> so that we can have less overloads and let users who only
> >>> want
> >>>> to
> >>>>>>>>>>>>> access
> >>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
> >>>>>> example:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>>>>>>>> globalKTable,
> >>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
> >>>> super
> >>>>>>>>> K, ?
> >>>>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>>>>>>>                                  final RichValueJoiner<?
> >>> super
> >>>> K,
> >>>>>>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Agreed. Fixed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
> >>> its
> >>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> - It was a typo. Fixed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 5. We need to move "RecordContext" from
> >>>> o.a.k.processor.internals
> >>>>>>>> to
> >>>>>>>>>>>>>>> o.a.k.processor.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>>>>>>>> ProcessorContext
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> RecordContext?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -
> >>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
> >>> maintenance
> >>>>>>>>> (both
> >>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
> >>>>>>>> methods),  I
> >>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
> >>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext and
> >> in
> >>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> >>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> >>>> final
> >>>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> My motivation is that for lambda expression in J8, users
> >> that
> >>>>>>>> would
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
> >>>> likely
> >>>>>>>> to
> >>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
> >>> the
> >>>>>>>> dummy
> >>>>>>>>>>> at
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
> >> like
> >>>> to
> >>>>>>>>> make
> >>>>>>>>>>>>> all
> >>>>>>>>>>>>>>> the "necessary" parameters prior to optional ones.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
> >> their
> >>>>>>>>> pairing
> >>>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> >> RichKeyValueMapper<?
> >>>>>>>> super
> >>>>>>>>> K,
> >>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>>>>>                                    final Serde<KR>
> >> keySerde,
> >>>>>>>>>>>>>>>                                    final Serde<V>
> >> valSerde);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>>>>>>>                                  final RichValueJoiner<?
> >>> super
> >>>> K,
> >>>>>>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> >>>> combo
> >>>>>>>> of
> >>>>>>>>>>>>> both
> >>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
> >> adder /
> >>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> >>> functions
> >>>>>>>> for
> >>>>>>>>>>>>> both;
> >>>>>>>>>>>>>>> so that we can have less overloads and let users who only
> >>> want
> >>>> to
> >>>>>>>>>>>>> access
> >>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
> >>>>>> example:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>>>>>>>> globalKTable,
> >>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
> >>>> super
> >>>>>>>>> K, ?
> >>>>>>>>>>>>>>> super
> >>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>>>>>>>                                  final RichValueJoiner<?
> >>> super
> >>>> K,
> >>>>>>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> >>>>>>>>> initializer,
> >>>>>>>>>>>>>>>                              final RichAggregator<? super
> >> K,
> >>> ?
> >>>>>>>> super
> >>>>>>>>>>> V,
> >>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>> aggregator,
> >>>>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
> >>>>>> deprecated
> >>>>>>>>> so
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>> not need to add its rich functions any more.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
> >> make
> >>>> its
> >>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> >>>> RichInitializer<VR,
> >>>>>>>> K>
> >>>>>>>>>>>>>>> initializer,
> >>>>>>>>>>>>>>>                                        final
> >> RichAggregator<?
> >>>>>>>> super
> >>>>>>>>> K,
> >>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>> super V, VR> aggregator);
> >>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> >>>> RichInitializer<VR,
> >>>>>>>> K>
> >>>>>>>>>>>>>>> initializer,
> >>>>>>>>>>>>>>>                                        final
> >> RichAggregator<?
> >>>>>>>> super
> >>>>>>>>> K,
> >>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>> super V, VR> aggregator,
> >>>>>>>>>>>>>>>                                        final
> >> Materialized<K,
> >>>> VR,
> >>>>>>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
> >>>> o.a.k.processor.internals
> >>>>>>>> to
> >>>>>>>>>>>>>>> o.a.k.processor.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>>>>>>>> ProcessorContext
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> RecordContext? Conceptually I think it would better staying
> >>> in
> >>>>>> the
> >>>>>>>>>>>>>>> ProcessorContext. Do you find this not doable in the
> >> internal
> >>>>>>>>>>>>>>> implementations?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <
> >> yuzhihong@gmail.com
> >>>>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>    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 <
> >>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 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 to KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >>>> initializer,
> >>>>>>>>>>>>>>>>>>                              final Aggregator<? super
> >> K, ?
> >>>>>>>> super
> >>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
> >>> in
> >>>>>> the
> >>>>>>>>>>>>>>> related
> >>>>>>>>>>>>>>>>> method. So, I had to overload all possible their Rich
> >>>>>>>>> counterparts:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
> >>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >>>> initializer,
> >>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
> >> ?
> >>>>>> super
> >>>>>>>>> V,
> >>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>>>>                              final RichAggregator<? super
> >>> K,
> >>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
> >>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >>>> initializer,
> >>>>>>>>>>>>>>>>>                              final RichAggregator<? super
> >>> K,
> >>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
> >> ?
> >>>>>> super
> >>>>>>>>> V,
> >>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // both adder and subtractor are rich
> >>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >>>> initializer,
> >>>>>>>>>>>>>>>>>                              final RichAggregator<? super
> >>> K,
> >>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>>>>                              final RichAggregator<? super
> >>> K,
> >>>> ?
> >>>>>>>>>>>>> super
> >>>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
> >> is
> >>>> not
> >>>>>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> this context");
> >>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> >>> the
> >>>> PR
> >>>>>>>> ?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> - I added some comments both inside and outside the code
> >>>>>>>> snippets
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>> Specifically, for the code snippet above, we add
> >> *commit()*
> >>>>>>>> method
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> *RecordContext* interface.
> >>>>>>>>>>>>>>>>> However, we want  *commit()* method to be used only for
> >>>>>>>>>>>>>> *RecordContext*
> >>>>>>>>>>>>>>>>> instances (at least for now), so we add
> >>>>>>>>>>>>> UnsupportedOperationException
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> all classes/interfaces that extend/implement
> >>> *RecordContext.*
> >>>>>>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
> >>>> within
> >>>>>>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
> >>>>>>>> required
> >>>>>>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
> >>> Rich
> >>>>>>>>>>>>>> interfaces
> >>>>>>>>>>>>>>>>> inside Processors.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
> >>> yuzhihong@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >>>>>> initializer,
> >>>>>>>>>>>>>>>>>>                              final Aggregator<? super
> >> K, ?
> >>>>>>>> super
> >>>>>>>>>>>>> V,
> >>>>>>>>>>>>>>> VR>
> >>>>>>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>   public RecordContext recordContext() {
> >>>>>>>>>>>>>>>>>>     return this.recordContext();
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
> >> is
> >>>> not
> >>>>>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> this context");
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> >>> the
> >>>> PR
> >>>>>>>> ?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Cheers
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 Jeyhun Karimov <
> >>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
> >>> KIP-159
> >>>> is
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> voted
> >>>>>>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as
> >> an
> >>>>>>>>>>>>>> "atomic"
> >>>>>>>>>>>>>>>>>> merge).
> >>>>>>>>>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there
> >> are
> >>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>> minor
> >>>>>>>>>>>>>>>>>>>>> updates
> >>>>>>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please
> >> correct
> >>> me
> >>>>>>>>>>>>> if
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>>>>>>>> or I
> >>>>>>>>>>>>>>>>>>>>>> misunderstood.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> >>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> >>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and
> >> vote
> >>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> >>>>>>>>>>>>>>>> yuzhihong@gmail.com
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> >>>>>>>>>>>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> >>>>>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>>>>>>> interfaces
> >>>>>>>>>>>>>>>>>>>>>>>>> into 1
> >>>>>>>>>>>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun
> >> Karimov <
> >>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> >>>>>>>>>>>>>>>>>> implements
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> >>>>>>>>>>>>>>> RichXXX
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>> XXXWithKey
> >>>>>>>>>>>>>>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> revert
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to this again.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> >>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> >>>>>>>>>>>>> RichXXXX
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>>>>>>>>>>>>> confluence/pages/viewpage.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> >>>>>>>>>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Matthias, Damian, all,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your comments and sorry for
> >>>>>>>>>>>>>>>> super-late
> >>>>>>>>>>>>>>>>>>>>> update.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sure, the DSL refactoring is not blocking
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I just re-read the KIP.

One minor comment: we don't need to introduce any deprecated methods.
Thus, RichValueTransformer#punctuate can be removed completely instead
of introducing it as deprecated.

Otherwise looks good to me.

Thanks for being so patient!


-Matthias

On 11/1/17 9:16 PM, Guozhang Wang wrote:
> Jeyhun,
> 
> I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
> carefully if we should add this functionality to the DSL layer moving
> forward since from what we discovered working on it the conclusion is that
> it would require revamping the public APIs quite a lot, and it's not clear
> if it is a good trade-off than asking users to call process() instead.
> 
> 
> Guozhang
> 
> 
> On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com> wrote:
> 
>> Hi Jeyhun, thanks, looks good.
>> Do we need to remove the line that says:
>>
>>    - on-demand commit() feature
>>
>> Cheers,
>> Damian
>>
>> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com> wrote:
>>
>>> Hi,
>>>
>>> I removed the 'commit()' feature, as we discussed. It simplified  the
>>> overall design of KIP a lot.
>>> If it is ok, I would like to start a VOTE thread.
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> Thanks. I understand what you are saying, but I don't agree that
>>>>
>>>>> but also we need a commit() method
>>>>
>>>> I would just not provide `commit()` at DSL level and close the
>>>> corresponding Jira as "not a problem" or similar.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
>>>>> Hi Matthias,
>>>>>
>>>>> Thanks for your comments. I agree that this is not the best way to
>> do.
>>> A
>>>>> bit of history behind this design.
>>>>>
>>>>> Prior doing this, I tried to provide ProcessorContext itself as an
>>>> argument
>>>>> in Rich interfaces. However, we dont want to give users that
>>> flexibility
>>>>> and “power”. Moreover, ProcessorContext contains processor level
>>>>> information and not Record level info. The only thing we need ij
>>>>> ProcessorContext is commit() method.
>>>>>
>>>>> So, as far as I understood, we need recor context (offset, timestamp
>>> and
>>>>> etc) but also we need a commit() method ( we dont want to provide
>>>>> ProcessorContext as a parameter so users can use
>>>> ProcessorContext.commit()
>>>>> ).
>>>>>
>>>>> As a result, I thought to “propagate” commit() call from
>> RecordContext
>>> to
>>>>> ProcessorContext() .
>>>>>
>>>>>
>>>>> If there is a misunderstanding in motvation/discussion of
>> KIP/included
>>>>> jiras please let me know.
>>>>>
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>>
>>>>> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <matthias@confluent.io
>>>
>>>> wrote:
>>>>>
>>>>>> I am personally still not convinced, that we should add `commit()`
>> at
>>>> all.
>>>>>>
>>>>>> @Guozhang: you created the original Jira. Can you elaborate a little
>>>>>> bit? Isn't requesting commits a low level API that should not be
>>> exposed
>>>>>> in the DSL? Just want to understand the motivation better. Why would
>>>>>> anybody that uses the DSL ever want to request a commit? To me,
>>>>>> requesting commits is useful if you manipulated state explicitly,
>> ie,
>>>>>> via Processor API.
>>>>>>
>>>>>> Also, for the solution: it seem rather unnatural to me, that we add
>>>>>> `commit()` to `RecordContext` -- from my understanding,
>>> `RecordContext`
>>>>>> is an helper object that provide access to record meta data.
>>> Requesting
>>>>>> a commit is something quite different. Additionally, a commit does
>> not
>>>>>> commit a specific record but a `RecrodContext` is for a specific
>>> record.
>>>>>>
>>>>>> To me, this does not seem to be a sound API design if we follow this
>>>> path.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>>>>>> Hi,
>>>>>>>
>>>>>>> Thanks for your suggestions.
>>>>>>>
>>>>>>> I have some comments, to make sure that there is no
>> misunderstanding.
>>>>>>>
>>>>>>>
>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>> enforce
>>>>>>>> user to consolidate this call as
>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>> implementation
>>>>>>>> of
>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>> changed
>>>> to
>>>>>>>> this call.
>>>>>>>
>>>>>>>
>>>>>>> - I think we should not deprecate `ProcessorContext.commit()`. The
>>> main
>>>>>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>>>>>> `RecordContext` is the one which is provided in Rich interfaces. So
>>> if
>>>>>> user
>>>>>>> wants to commit, then there should be some method inside
>>>> `RecordContext`
>>>>>> to
>>>>>>> do so. Internally, `RecordContext.commit()` calls
>>>>>>> `ProcessorContext.commit()`  (see the last code snippet in
>> KIP-159):
>>>>>>>
>>>>>>> @Override
>>>>>>>     public void process(final K1 key, final V1 value) {
>>>>>>>
>>>>>>>         recordContext = new RecordContext() {               //
>>>>>>> recordContext initialization is added in this KIP
>>>>>>>             @Override
>>>>>>>             public void commit() {
>>>>>>>                 context().commit();
>>>>>>>             }
>>>>>>>
>>>>>>>             @Override
>>>>>>>             public long offset() {
>>>>>>>                 return context().recordContext().offset();
>>>>>>>             }
>>>>>>>
>>>>>>>             @Override
>>>>>>>             public long timestamp() {
>>>>>>>                 return context().recordContext().timestamp();
>>>>>>>             }
>>>>>>>
>>>>>>>             @Override
>>>>>>>             public String topic() {
>>>>>>>                 return context().recordContext().topic();
>>>>>>>             }
>>>>>>>
>>>>>>>             @Override
>>>>>>>             public int partition() {
>>>>>>>                 return context().recordContext().partition();
>>>>>>>             }
>>>>>>>       };
>>>>>>>
>>>>>>>
>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case
>> IMO.
>>>>>>>
>>>>>>>
>>>>>>> 2. Add the `task` reference to the impl class,
>>>> `ProcessorRecordContext`,
>>>>>> so
>>>>>>>> that it can implement the commit call itself.
>>>>>>>
>>>>>>>
>>>>>>> - Actually, I don't think that we need `commit()` in
>>>>>>> `ProcessorRecordContext`. 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 `RecordContext()` just
>>>> only
>>>>>> to
>>>>>>> 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 KIP, we made `RecordContext` to be the parent class of
>>>>>>> `ProcessorRecordContext`, just because of they share quite amount
>> of
>>>>>>> methods and it is logical to enable inheritance between those two.
>>>>>>>
>>>>>>> 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
>>> snippet
>>>>>>>> below 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 has gone though many
>>>> changes
>>>>>>> based on user/developer needs, both in
>>>>>> documentation-/implementation-wise.
>>>>>>>
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Jeyhun
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks for the information Jeyhun. I had also forgot about
>>> KAFKA-3907
>>>>>> with
>>>>>>>> this 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:
>>>>>>>>
>>>>>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
>>>> enforce
>>>>>>>> user to consolidate this call as
>>>>>>>> "processorContext.recordContext().commit()". And internal
>>>> implementation
>>>>>>>> of
>>>>>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
>>> changed
>>>> to
>>>>>>>> this call.
>>>>>>>>
>>>>>>>> 2. Add the `task` reference to the impl class,
>>>>>> `ProcessorRecordContext`, so
>>>>>>>> that it can implement the commit call itself.
>>>>>>>>
>>>>>>>> 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
>>> snippet
>>>>>>>> below would need to be updated as well.
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
>>>> matthias@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.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -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
>>>>>>>> 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/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. Sax <
>>>>>> matthias@confluent.io
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 DSL users should not worry about
>> --
>>>> and
>>>>>>>> if
>>>>>>>>>>> one really needs this, an advanced user can still insert a
>> dummy
>>>>>>>>>>> `transform` to request a commit from there.
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>>>>>> 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
>>> argument
>>>> to
>>>>>>>>> Rich
>>>>>>>>>>>> methods:
>>>>>>>>>>>>
>>>>>>>>>>>> instead of
>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>     VR apply(final V value,
>>>>>>>>>>>>              final K key,
>>>>>>>>>>>>              final RecordContext recordContext);
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> we can adopt
>>>>>>>>>>>>
>>>>>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>>>>>     VR apply(final V value,
>>>>>>>>>>>>              final K key,
>>>>>>>>>>>>              final RecordContext recordContext,
>>>>>>>>>>>>              final ProcessorContext processorContext);
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> However, in this case, a user can get confused as
>>> ProcessorContext
>>>>>>>> and
>>>>>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
>>> wangguoz@gmail.com
>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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 whole task up to this processed record, not only that
>>>> single
>>>>>>>>>>> record
>>>>>>>>>>>>> itself.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
>>>> final
>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> - Fixed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>> their
>>>>>>>> pairing
>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>> RichKeyValueMapper<?
>>>>>>>> super
>>>>>>>>> K,
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>                                    final Serde<KR>
>> keySerde,
>>>>>>>>>>>>>>>                                    final Serde<V>
>> valSerde);
>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>> super
>>>> K,
>>>>>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Fixed
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>> combo
>>>>>>>> of
>>>>>>>>>>> both
>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>> adder /
>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>> functions
>>>>>>>> for
>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>> want
>>>> to
>>>>>>>>>>>>> access
>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>> example:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
>>>> super
>>>>>>>>> K, ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>> super
>>>> K,
>>>>>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
>>> its
>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>> o.a.k.processor.internals
>>>>>>>> to
>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>> ProcessorContext
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> RecordContext?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -
>>>>>>>>>>>>>> Because it makes sense logically and  to reduce code
>>> maintenance
>>>>>>>>> (both
>>>>>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>>>>>> methods),  I
>>>>>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>>>>>> Since we need commit() method both in ProcessorContext and
>> in
>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
>>>> final
>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> My motivation is that for lambda expression in J8, users
>> that
>>>>>>>> would
>>>>>>>>>>> not
>>>>>>>>>>>>>>> care about the key but only the context, or vice versa, is
>>>> likely
>>>>>>>> to
>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
>>> the
>>>>>>>> dummy
>>>>>>>>>>> at
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
>> like
>>>> to
>>>>>>>>> make
>>>>>>>>>>>>> all
>>>>>>>>>>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. Some of the listed functions are not necessary since
>> their
>>>>>>>>> pairing
>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
>> RichKeyValueMapper<?
>>>>>>>> super
>>>>>>>>> K,
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>>>>>                                    final Serde<KR>
>> keySerde,
>>>>>>>>>>>>>>>                                    final Serde<V>
>> valSerde);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>> super
>>>> K,
>>>>>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
>>>> combo
>>>>>>>> of
>>>>>>>>>>>>> both
>>>>>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
>> adder /
>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
>>> functions
>>>>>>>> for
>>>>>>>>>>>>> both;
>>>>>>>>>>>>>>> so that we can have less overloads and let users who only
>>> want
>>>> to
>>>>>>>>>>>>> access
>>>>>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>>>>>> example:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>>>>>> globalKTable,
>>>>>>>>>>>>>>>                                  final RichKeyValueMapper<?
>>>> super
>>>>>>>>> K, ?
>>>>>>>>>>>>>>> super
>>>>>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>>>>>                                  final RichValueJoiner<?
>>> super
>>>> K,
>>>>>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
>>>>>>>>> initializer,
>>>>>>>>>>>>>>>                              final RichAggregator<? super
>> K,
>>> ?
>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>> aggregator,
>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
>>>>>> deprecated
>>>>>>>>> so
>>>>>>>>>>>>> we
>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>> not need to add its rich functions any more.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
>> make
>>>> its
>>>>>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>> RichInitializer<VR,
>>>>>>>> K>
>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>                                        final
>> RichAggregator<?
>>>>>>>> super
>>>>>>>>> K,
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super V, VR> aggregator);
>>>>>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
>>>> RichInitializer<VR,
>>>>>>>> K>
>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>                                        final
>> RichAggregator<?
>>>>>>>> super
>>>>>>>>> K,
>>>>>>>>>>>>> ?
>>>>>>>>>>>>>>> super V, VR> aggregator,
>>>>>>>>>>>>>>>                                        final
>> Materialized<K,
>>>> VR,
>>>>>>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 5. We need to move "RecordContext" from
>>>> o.a.k.processor.internals
>>>>>>>> to
>>>>>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>>>>>> ProcessorContext
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> RecordContext? Conceptually I think it would better staying
>>> in
>>>>>> the
>>>>>>>>>>>>>>> ProcessorContext. Do you find this not doable in the
>> internal
>>>>>>>>>>>>>>> implementations?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <
>> yuzhihong@gmail.com
>>>>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>    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 <
>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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 to KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>> initializer,
>>>>>>>>>>>>>>>>>>                              final Aggregator<? super
>> K, ?
>>>>>>>> super
>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
>>> in
>>>>>> the
>>>>>>>>>>>>>>> related
>>>>>>>>>>>>>>>>> method. So, I had to overload all possible their Rich
>>>>>>>>> counterparts:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>> initializer,
>>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
>> ?
>>>>>> super
>>>>>>>>> V,
>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>> K,
>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>> initializer,
>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>> K,
>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>                              final Aggregator<? super K,
>> ?
>>>>>> super
>>>>>>>>> V,
>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // both adder and subtractor are rich
>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>> initializer,
>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>> K,
>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>                              final RichAggregator<? super
>>> K,
>>>> ?
>>>>>>>>>>>>> super
>>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
>> is
>>>> not
>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>> the
>>>> PR
>>>>>>>> ?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - I added some comments both inside and outside the code
>>>>>>>> snippets
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>> Specifically, for the code snippet above, we add
>> *commit()*
>>>>>>>> method
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> *RecordContext* interface.
>>>>>>>>>>>>>>>>> However, we want  *commit()* method to be used only for
>>>>>>>>>>>>>> *RecordContext*
>>>>>>>>>>>>>>>>> instances (at least for now), so we add
>>>>>>>>>>>>> UnsupportedOperationException
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> all classes/interfaces that extend/implement
>>> *RecordContext.*
>>>>>>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
>>>> within
>>>>>>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
>>>>>>>> required
>>>>>>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
>>> Rich
>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>> inside Processors.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
>>> yuzhihong@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>>>>>> initializer,
>>>>>>>>>>>>>>>>>>                              final Aggregator<? super
>> K, ?
>>>>>>>> super
>>>>>>>>>>>>> V,
>>>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>   public RecordContext recordContext() {
>>>>>>>>>>>>>>>>>>     return this.recordContext();
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
>> is
>>>> not
>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Is the exception going to be replaced with real code in
>>> the
>>>> PR
>>>>>>>> ?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 Jeyhun Karimov <
>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
>>> KIP-159
>>>> is
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> voted
>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as
>> an
>>>>>>>>>>>>>> "atomic"
>>>>>>>>>>>>>>>>>> merge).
>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there
>> are
>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please
>> correct
>>> me
>>>>>>>>>>>>> if
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>> or I
>>>>>>>>>>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and
>> vote
>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>>>>>>>>>>> yuzhihong@gmail.com
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun
>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>>>>>>>>>>> RichXXX
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> revert
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>>>>>>>>>>> RichXXXX
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>>>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
>>>>>>>>>>>>>> issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
>>>>>>>>>>>>>> Guy
>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overrides.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
>>>>>>>>>>>>>> RecordContext,
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>> right.
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> full
>>>>>>>>>>>>>>>>>>>>> picture
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>> parts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
>>>>>>>>>>>>> as
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> thought.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>> Guy
>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>
> 
> 
> 


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

Posted by Guozhang Wang <wa...@gmail.com>.
Jeyhun,

I think I'm convinced to not do KAFKA-3907 in this KIP. We should think
carefully if we should add this functionality to the DSL layer moving
forward since from what we discovered working on it the conclusion is that
it would require revamping the public APIs quite a lot, and it's not clear
if it is a good trade-off than asking users to call process() instead.


Guozhang


On Wed, Nov 1, 2017 at 4:50 AM, Damian Guy <da...@gmail.com> wrote:

> Hi Jeyhun, thanks, looks good.
> Do we need to remove the line that says:
>
>    - on-demand commit() feature
>
> Cheers,
> Damian
>
> On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com> wrote:
>
> > Hi,
> >
> > I removed the 'commit()' feature, as we discussed. It simplified  the
> > overall design of KIP a lot.
> > If it is ok, I would like to start a VOTE thread.
> >
> > Cheers,
> > Jeyhun
> >
> > On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > Thanks. I understand what you are saying, but I don't agree that
> > >
> > > > but also we need a commit() method
> > >
> > > I would just not provide `commit()` at DSL level and close the
> > > corresponding Jira as "not a problem" or similar.
> > >
> > >
> > > -Matthias
> > >
> > > On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> > > > Hi Matthias,
> > > >
> > > > Thanks for your comments. I agree that this is not the best way to
> do.
> > A
> > > > bit of history behind this design.
> > > >
> > > > Prior doing this, I tried to provide ProcessorContext itself as an
> > > argument
> > > > in Rich interfaces. However, we dont want to give users that
> > flexibility
> > > > and “power”. Moreover, ProcessorContext contains processor level
> > > > information and not Record level info. The only thing we need ij
> > > > ProcessorContext is commit() method.
> > > >
> > > > So, as far as I understood, we need recor context (offset, timestamp
> > and
> > > > etc) but also we need a commit() method ( we dont want to provide
> > > > ProcessorContext as a parameter so users can use
> > > ProcessorContext.commit()
> > > > ).
> > > >
> > > > As a result, I thought to “propagate” commit() call from
> RecordContext
> > to
> > > > ProcessorContext() .
> > > >
> > > >
> > > > If there is a misunderstanding in motvation/discussion of
> KIP/included
> > > > jiras please let me know.
> > > >
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > >
> > > > On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <matthias@confluent.io
> >
> > > wrote:
> > > >
> > > >> I am personally still not convinced, that we should add `commit()`
> at
> > > all.
> > > >>
> > > >> @Guozhang: you created the original Jira. Can you elaborate a little
> > > >> bit? Isn't requesting commits a low level API that should not be
> > exposed
> > > >> in the DSL? Just want to understand the motivation better. Why would
> > > >> anybody that uses the DSL ever want to request a commit? To me,
> > > >> requesting commits is useful if you manipulated state explicitly,
> ie,
> > > >> via Processor API.
> > > >>
> > > >> Also, for the solution: it seem rather unnatural to me, that we add
> > > >> `commit()` to `RecordContext` -- from my understanding,
> > `RecordContext`
> > > >> is an helper object that provide access to record meta data.
> > Requesting
> > > >> a commit is something quite different. Additionally, a commit does
> not
> > > >> commit a specific record but a `RecrodContext` is for a specific
> > record.
> > > >>
> > > >> To me, this does not seem to be a sound API design if we follow this
> > > path.
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>
> > > >>
> > > >> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> > > >>> Hi,
> > > >>>
> > > >>> Thanks for your suggestions.
> > > >>>
> > > >>> I have some comments, to make sure that there is no
> misunderstanding.
> > > >>>
> > > >>>
> > > >>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> > > enforce
> > > >>>> user to consolidate this call as
> > > >>>> "processorContext.recordContext().commit()". And internal
> > > implementation
> > > >>>> of
> > > >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> > changed
> > > to
> > > >>>> this call.
> > > >>>
> > > >>>
> > > >>> - I think we should not deprecate `ProcessorContext.commit()`. The
> > main
> > > >>> intuition that we introduce `commit()` in `RecordContext` is that,
> > > >>> `RecordContext` is the one which is provided in Rich interfaces. So
> > if
> > > >> user
> > > >>> wants to commit, then there should be some method inside
> > > `RecordContext`
> > > >> to
> > > >>> do so. Internally, `RecordContext.commit()` calls
> > > >>> `ProcessorContext.commit()`  (see the last code snippet in
> KIP-159):
> > > >>>
> > > >>> @Override
> > > >>>     public void process(final K1 key, final V1 value) {
> > > >>>
> > > >>>         recordContext = new RecordContext() {               //
> > > >>> recordContext initialization is added in this KIP
> > > >>>             @Override
> > > >>>             public void commit() {
> > > >>>                 context().commit();
> > > >>>             }
> > > >>>
> > > >>>             @Override
> > > >>>             public long offset() {
> > > >>>                 return context().recordContext().offset();
> > > >>>             }
> > > >>>
> > > >>>             @Override
> > > >>>             public long timestamp() {
> > > >>>                 return context().recordContext().timestamp();
> > > >>>             }
> > > >>>
> > > >>>             @Override
> > > >>>             public String topic() {
> > > >>>                 return context().recordContext().topic();
> > > >>>             }
> > > >>>
> > > >>>             @Override
> > > >>>             public int partition() {
> > > >>>                 return context().recordContext().partition();
> > > >>>             }
> > > >>>       };
> > > >>>
> > > >>>
> > > >>> So, we cannot deprecate `ProcessorContext.commit()` in this case
> IMO.
> > > >>>
> > > >>>
> > > >>> 2. Add the `task` reference to the impl class,
> > > `ProcessorRecordContext`,
> > > >> so
> > > >>>> that it can implement the commit call itself.
> > > >>>
> > > >>>
> > > >>> - Actually, I don't think that we need `commit()` in
> > > >>> `ProcessorRecordContext`. 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 `RecordContext()` just
> > > only
> > > >> to
> > > >>> 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 KIP, we made `RecordContext` to be the parent class of
> > > >>> `ProcessorRecordContext`, just because of they share quite amount
> of
> > > >>> methods and it is logical to enable inheritance between those two.
> > > >>>
> > > >>> 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
> > snippet
> > > >>>> below 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 has gone though many
> > > changes
> > > >>> based on user/developer needs, both in
> > > >> documentation-/implementation-wise.
> > > >>>
> > > >>>
> > > >>> Cheers,
> > > >>> Jeyhun
> > > >>>
> > > >>>
> > > >>>
> > > >>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
> > > >> wrote:
> > > >>>
> > > >>>> Thanks for the information Jeyhun. I had also forgot about
> > KAFKA-3907
> > > >> with
> > > >>>> this 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:
> > > >>>>
> > > >>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> > > enforce
> > > >>>> user to consolidate this call as
> > > >>>> "processorContext.recordContext().commit()". And internal
> > > implementation
> > > >>>> of
> > > >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> > changed
> > > to
> > > >>>> this call.
> > > >>>>
> > > >>>> 2. Add the `task` reference to the impl class,
> > > >> `ProcessorRecordContext`, so
> > > >>>> that it can implement the commit call itself.
> > > >>>>
> > > >>>> 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
> > snippet
> > > >>>> below would need to be updated as well.
> > > >>>>
> > > >>>>
> > > >>>> Guozhang
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> > > matthias@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.
> > > >>>>>
> > > >>>>>
> > > >>>>> -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
> > > >>>> 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/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. Sax <
> > > >> matthias@confluent.io
> > > >>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>> 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 DSL users should not worry about
> --
> > > and
> > > >>>> if
> > > >>>>>>> one really needs this, an advanced user can still insert a
> dummy
> > > >>>>>>> `transform` to request a commit from there.
> > > >>>>>>>
> > > >>>>>>> -Matthias
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> > > >>>>>>>> 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
> > argument
> > > to
> > > >>>>> Rich
> > > >>>>>>>> methods:
> > > >>>>>>>>
> > > >>>>>>>> instead of
> > > >>>>>>>> public interface RichValueMapper<V, VR, K> {
> > > >>>>>>>>     VR apply(final V value,
> > > >>>>>>>>              final K key,
> > > >>>>>>>>              final RecordContext recordContext);
> > > >>>>>>>> }
> > > >>>>>>>>
> > > >>>>>>>> we can adopt
> > > >>>>>>>>
> > > >>>>>>>> public interface RichValueMapper<V, VR, K> {
> > > >>>>>>>>     VR apply(final V value,
> > > >>>>>>>>              final K key,
> > > >>>>>>>>              final RecordContext recordContext,
> > > >>>>>>>>              final ProcessorContext processorContext);
> > > >>>>>>>> }
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> However, in this case, a user can get confused as
> > ProcessorContext
> > > >>>> and
> > > >>>>>>>> RecordContext share some methods with the same name.
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> Cheers,
> > > >>>>>>>> Jeyhun
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > >>>>>>> wrote:
> > > >>>>>>>>
> > > >>>>>>>>> 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 whole task up to this processed record, not only that
> > > single
> > > >>>>>>> record
> > > >>>>>>>>> itself.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> Guozhang
> > > >>>>>>>>>
> > > >>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> > > >>>> je.karimov@gmail.com
> > > >>>>>>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> 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<V1, V2, VR, K> {
> > > >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> > > final
> > > >>>>>>>>>>> RecordContext
> > > >>>>>>>>>>> recordContext);
> > > >>>>>>>>>>> }
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> - Fixed.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 2. Some of the listed functions are not necessary since
> their
> > > >>>> pairing
> > > >>>>>>>>> APIs
> > > >>>>>>>>>>> are being deprecated in 1.0 already:
> > > >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> RichKeyValueMapper<?
> > > >>>> super
> > > >>>>> K,
> > > >>>>>>>>> ?
> > > >>>>>>>>>>> super V, KR> selector,
> > > >>>>>>>>>>>                                    final Serde<KR>
> keySerde,
> > > >>>>>>>>>>>                                    final Serde<V>
> valSerde);
> > > >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > > >>>>>>>>>>>                                  final RichValueJoiner<?
> > super
> > > K,
> > > >>>> ?
> > > >>>>>>>>> super
> > > >>>>>>>>>>> V,
> > > >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> > > >>>>>>>>>>>                                  final Serde<K> keySerde,
> > > >>>>>>>>>>>                                  final Serde<V> valSerde);
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> -Fixed
> > > >>>>>>>>>>
> > > >>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> > > combo
> > > >>>> of
> > > >>>>>>> both
> > > >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
> adder /
> > > >>>>>>>>> subtractor,
> > > >>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> > functions
> > > >>>> for
> > > >>>>>>>>> both;
> > > >>>>>>>>>>> so that we can have less overloads and let users who only
> > want
> > > to
> > > >>>>>>>>> access
> > > >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> > > >> example:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > > >>>>>>>>> globalKTable,
> > > >>>>>>>>>>>                                  final RichKeyValueMapper<?
> > > super
> > > >>>>> K, ?
> > > >>>>>>>>>>> super
> > > >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> > > >>>>>>>>>>>                                  final RichValueJoiner<?
> > super
> > > K,
> > > >>>> ?
> > > >>>>>>>>> super
> > > >>>>>>>>>>> V,
> > > >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> -Agreed. Fixed.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
> > its
> > > >>>>>>>>>>> Initializer also "rich" functions? I.e.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> - It was a typo. Fixed.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 5. We need to move "RecordContext" from
> > > o.a.k.processor.internals
> > > >>>> to
> > > >>>>>>>>>>> o.a.k.processor.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> > > >>>>> ProcessorContext
> > > >>>>>>>>> to
> > > >>>>>>>>>>> RecordContext?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> -
> > > >>>>>>>>>> Because it makes sense logically and  to reduce code
> > maintenance
> > > >>>>> (both
> > > >>>>>>>>>> interfaces have offset() timestamp() topic() partition()
> > > >>>> methods),  I
> > > >>>>>>>>>> inherit ProcessorContext from RecordContext.
> > > >>>>>>>>>> Since we need commit() method both in ProcessorContext and
> in
> > > >>>>>>>>> RecordContext
> > > >>>>>>>>>> I move commit() method to parent class (RecordContext).
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> Cheers,
> > > >>>>>>>>>> Jeyhun
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> > > >>>> wangguoz@gmail.com>
> > > >>>>>>>>>> wrote:
> > > >>>>>>>>>>
> > > >>>>>>>>>>> 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<V1, V2, VR, K> {
> > > >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> > > final
> > > >>>>>>>>>>> RecordContext
> > > >>>>>>>>>>> recordContext);
> > > >>>>>>>>>>> }
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> My motivation is that for lambda expression in J8, users
> that
> > > >>>> would
> > > >>>>>>> not
> > > >>>>>>>>>>> care about the key but only the context, or vice versa, is
> > > likely
> > > >>>> to
> > > >>>>>>>>>> write
> > > >>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
> > the
> > > >>>> dummy
> > > >>>>>>> at
> > > >>>>>>>>>> the
> > > >>>>>>>>>>> beginning of the parameter list. Generally speaking we'd
> like
> > > to
> > > >>>>> make
> > > >>>>>>>>> all
> > > >>>>>>>>>>> the "necessary" parameters prior to optional ones.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 2. Some of the listed functions are not necessary since
> their
> > > >>>>> pairing
> > > >>>>>>>>>> APIs
> > > >>>>>>>>>>> are being deprecated in 1.0 already:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final
> RichKeyValueMapper<?
> > > >>>> super
> > > >>>>> K,
> > > >>>>>>>>> ?
> > > >>>>>>>>>>> super V, KR> selector,
> > > >>>>>>>>>>>                                    final Serde<KR>
> keySerde,
> > > >>>>>>>>>>>                                    final Serde<V>
> valSerde);
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > > >>>>>>>>>>>                                  final RichValueJoiner<?
> > super
> > > K,
> > > >>>> ?
> > > >>>>>>>>> super
> > > >>>>>>>>>>> V,
> > > >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> > > >>>>>>>>>>>                                  final Serde<K> keySerde,
> > > >>>>>>>>>>>                                  final Serde<V> valSerde);
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> > > combo
> > > >>>> of
> > > >>>>>>>>> both
> > > >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or
> adder /
> > > >>>>>>>>> subtractor,
> > > >>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> > functions
> > > >>>> for
> > > >>>>>>>>> both;
> > > >>>>>>>>>>> so that we can have less overloads and let users who only
> > want
> > > to
> > > >>>>>>>>> access
> > > >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> > > >> example:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > > >>>>>>>>> globalKTable,
> > > >>>>>>>>>>>                                  final RichKeyValueMapper<?
> > > super
> > > >>>>> K, ?
> > > >>>>>>>>>>> super
> > > >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> > > >>>>>>>>>>>                                  final RichValueJoiner<?
> > super
> > > K,
> > > >>>> ?
> > > >>>>>>>>> super
> > > >>>>>>>>>>> V,
> > > >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> > > >>>>> initializer,
> > > >>>>>>>>>>>                              final RichAggregator<? super
> K,
> > ?
> > > >>>> super
> > > >>>>>>> V,
> > > >>>>>>>>>> VR>
> > > >>>>>>>>>>> aggregator,
> > > >>>>>>>>>>>                              final Materialized<K, VR,
> > > >>>>>>>>>> KeyValueStore<Bytes,
> > > >>>>>>>>>>> byte[]>> materialized);
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
> > > >> deprecated
> > > >>>>> so
> > > >>>>>>>>> we
> > > >>>>>>>>>> do
> > > >>>>>>>>>>> not need to add its rich functions any more.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not
> make
> > > its
> > > >>>>>>>>>>> Initializer also "rich" functions? I.e.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> > > RichInitializer<VR,
> > > >>>> K>
> > > >>>>>>>>>>> initializer,
> > > >>>>>>>>>>>                                        final
> RichAggregator<?
> > > >>>> super
> > > >>>>> K,
> > > >>>>>>>>> ?
> > > >>>>>>>>>>> super V, VR> aggregator);
> > > >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> > > RichInitializer<VR,
> > > >>>> K>
> > > >>>>>>>>>>> initializer,
> > > >>>>>>>>>>>                                        final
> RichAggregator<?
> > > >>>> super
> > > >>>>> K,
> > > >>>>>>>>> ?
> > > >>>>>>>>>>> super V, VR> aggregator,
> > > >>>>>>>>>>>                                        final
> Materialized<K,
> > > VR,
> > > >>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 5. We need to move "RecordContext" from
> > > o.a.k.processor.internals
> > > >>>> to
> > > >>>>>>>>>>> o.a.k.processor.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> > > >>>>> ProcessorContext
> > > >>>>>>>>> to
> > > >>>>>>>>>>> RecordContext? Conceptually I think it would better staying
> > in
> > > >> the
> > > >>>>>>>>>>> ProcessorContext. Do you find this not doable in the
> internal
> > > >>>>>>>>>>> implementations?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Guozhang
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <
> yuzhihong@gmail.com
> > >
> > > >>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>>    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 <
> > > >>>>>>>>> je.karimov@gmail.com
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>> 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 to KIP.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > > initializer,
> > > >>>>>>>>>>>>>>                              final Aggregator<? super
> K, ?
> > > >>>> super
> > > >>>>>>>>> V,
> > > >>>>>>>>>>> VR>
> > > >>>>>>>>>>>>>> adder,
> > > >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
> > in
> > > >> the
> > > >>>>>>>>>>> related
> > > >>>>>>>>>>>>> method. So, I had to overload all possible their Rich
> > > >>>>> counterparts:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
> > > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > > initializer,
> > > >>>>>>>>>>>>>                              final Aggregator<? super K,
> ?
> > > >> super
> > > >>>>> V,
> > > >>>>>>>>>> VR>
> > > >>>>>>>>>>>>> adder,
> > > >>>>>>>>>>>>>                              final RichAggregator<? super
> > K,
> > > ?
> > > >>>>>>>>> super
> > > >>>>>>>>>> V,
> > > >>>>>>>>>>>> VR>
> > > >>>>>>>>>>>>> subtractor,
> > > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > > >>>>>>>>>>>> KeyValueStore<Bytes,
> > > >>>>>>>>>>>>> byte[]>> materialized);
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
> > > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > > initializer,
> > > >>>>>>>>>>>>>                              final RichAggregator<? super
> > K,
> > > ?
> > > >>>>>>>>> super
> > > >>>>>>>>>> V,
> > > >>>>>>>>>>>> VR>
> > > >>>>>>>>>>>>> adder,
> > > >>>>>>>>>>>>>                              final Aggregator<? super K,
> ?
> > > >> super
> > > >>>>> V,
> > > >>>>>>>>>> VR>
> > > >>>>>>>>>>>>> subtractor,
> > > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > > >>>>>>>>>>>> KeyValueStore<Bytes,
> > > >>>>>>>>>>>>> byte[]>> materialized);
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> // both adder and subtractor are rich
> > > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > > initializer,
> > > >>>>>>>>>>>>>                              final RichAggregator<? super
> > K,
> > > ?
> > > >>>>>>>>> super
> > > >>>>>>>>>> V,
> > > >>>>>>>>>>>> VR>
> > > >>>>>>>>>>>>> adder,
> > > >>>>>>>>>>>>>                              final RichAggregator<? super
> > K,
> > > ?
> > > >>>>>>>>> super
> > > >>>>>>>>>> V,
> > > >>>>>>>>>>>> VR>
> > > >>>>>>>>>>>>> subtractor,
> > > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > > >>>>>>>>>>>> KeyValueStore<Bytes,
> > > >>>>>>>>>>>>> byte[]>> materialized);
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> > > >>>>>>>>>>>>>>    void commit () {
> > > >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
> is
> > > not
> > > >>>>>>>>>>>> supported
> > > >>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>> this context");
> > > >>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> > the
> > > PR
> > > >>>> ?
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> - I added some comments both inside and outside the code
> > > >>>> snippets
> > > >>>>>>>>> in
> > > >>>>>>>>>>> KIP.
> > > >>>>>>>>>>>>> Specifically, for the code snippet above, we add
> *commit()*
> > > >>>> method
> > > >>>>>>>>> to
> > > >>>>>>>>>>>>> *RecordContext* interface.
> > > >>>>>>>>>>>>> However, we want  *commit()* method to be used only for
> > > >>>>>>>>>> *RecordContext*
> > > >>>>>>>>>>>>> instances (at least for now), so we add
> > > >>>>>>>>> UnsupportedOperationException
> > > >>>>>>>>>>> in
> > > >>>>>>>>>>>>> all classes/interfaces that extend/implement
> > *RecordContext.*
> > > >>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
> > > within
> > > >>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
> > > >>>> required
> > > >>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
> > Rich
> > > >>>>>>>>>> interfaces
> > > >>>>>>>>>>>>> inside Processors.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
> > yuzhihong@gmail.com>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 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.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > > >> initializer,
> > > >>>>>>>>>>>>>>                              final Aggregator<? super
> K, ?
> > > >>>> super
> > > >>>>>>>>> V,
> > > >>>>>>>>>>> VR>
> > > >>>>>>>>>>>>>> adder,
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>   public RecordContext recordContext() {
> > > >>>>>>>>>>>>>>     return this.recordContext();
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>    void commit () {
> > > >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit()
> is
> > > not
> > > >>>>>>>>>>>> supported
> > > >>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>> this context");
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> > the
> > > PR
> > > >>>> ?
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Cheers
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> > > >>>>>>>>>>> je.karimov@gmail.com>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 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 <
> > > >>>>>>>>>>>> je.karimov@gmail.com>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> 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 <
> > > >>>>>>>>>> damian.guy@gmail.com
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 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 Jeyhun Karimov <
> > > >>>>>>>>>>> je.karimov@gmail.com
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
> > KIP-159
> > > is
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>>> voted
> > > >>>>>>>>>>>>>>> yet
> > > >>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as
> an
> > > >>>>>>>>>> "atomic"
> > > >>>>>>>>>>>>>> merge).
> > > >>>>>>>>>>>>>>>>> So
> > > >>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there
> are
> > > >>>>>>>>> some
> > > >>>>>>>>>>>> minor
> > > >>>>>>>>>>>>>>>>> updates
> > > >>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please
> correct
> > me
> > > >>>>>>>>> if
> > > >>>>>>>>>> I
> > > >>>>>>>>>>> am
> > > >>>>>>>>>>>>>> wrong
> > > >>>>>>>>>>>>>>>>> or I
> > > >>>>>>>>>>>>>>>>>> misunderstood.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > > >>>>>>>>>>>> damian.guy@gmail.com>
> > > >>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> +1
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > > >>>>>>>>>>>> wangguoz@gmail.com>
> > > >>>>>>>>>>>>>>>>> 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
> > > >>>>>>>>>> would
> > > >>>>>>>>>>>>> still
> > > >>>>>>>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and
> vote
> > > >>>>>>>>> on.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Guozhang
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > > >>>>>>>>>>>> yuzhihong@gmail.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> +1
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > > >>>>>>>>>>>>>> bbejeck@gmail.com
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> > > >>>>>>>>>>>> ValueXXXXWithKey
> > > >>>>>>>>>>>>>>>>>> interfaces
> > > >>>>>>>>>>>>>>>>>>>>> into 1
> > > >>>>>>>>>>>>>>>>>>>>>> interface.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>> Bill
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun
> Karimov <
> > > >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 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
> > > >>>>>>>>> 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
> > > >>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> > > >>>>>>>>>>>>>> implements
> > > >>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> > > >>>>>>>>>>>>>>>>>>>>>>> ......
> > > >>>>>>>>>>>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> > > >>>>>>>>>>> RichXXX
> > > >>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>> XXXWithKey
> > > >>>>>>>>>>>>>>>>>>>>>> inside
> > > >>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> revert
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>>>> design
> > > >>>>>>>>>>>>>>>>>>>>>>> to this again.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > >>>>>>>>>>>>>>>>> damian.guy@gmail.com
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> > > >>>>>>>>> was
> > > >>>>>>>>>>>>>> thinking
> > > >>>>>>>>>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>> along
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> > > >>>>>>>>> RichXXXX
> > > >>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
> > > >>>>>>>>>>>>>>>>>>>>>> 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?
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>> Damian
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> [1]
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > > >>>>>>>>>>> confluence/pages/viewpage.
> > > >>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> > > >>>>>>>>>> Karimov <
> > > >>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
> > > >>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> > > >>>>>>>>>> issue.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> > > >>>>>>>>>> Guy
> > > >>>>>>>>>>> <
> > > >>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > > >>>>>>>>>>>>>>>>>>>>>>>>> overrides.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> > > >>>>>>>>>> RecordContext,
> > > >>>>>>>>>>>> you
> > > >>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>> right.
> > > >>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> full
> > > >>>>>>>>>>>>>>>>> picture
> > > >>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>> parts
> > > >>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> > > >>>>>>>>> as
> > > >>>>>>>>>> I
> > > >>>>>>>>>>>>>> thought.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> > > >>>>>>>>> Damian
> > > >>>>>>>>>>> Guy
> > > >>>>>>>>>>>> <
> > > >>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > > >>>>>>>>>>>>>>>>>>>>>>>>
>



-- 
-- Guozhang

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

Posted by Damian Guy <da...@gmail.com>.
Hi Jeyhun, thanks, looks good.
Do we need to remove the line that says:

   - on-demand commit() feature

Cheers,
Damian

On Tue, 31 Oct 2017 at 23:07 Jeyhun Karimov <je...@gmail.com> wrote:

> Hi,
>
> I removed the 'commit()' feature, as we discussed. It simplified  the
> overall design of KIP a lot.
> If it is ok, I would like to start a VOTE thread.
>
> Cheers,
> Jeyhun
>
> On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > Thanks. I understand what you are saying, but I don't agree that
> >
> > > but also we need a commit() method
> >
> > I would just not provide `commit()` at DSL level and close the
> > corresponding Jira as "not a problem" or similar.
> >
> >
> > -Matthias
> >
> > On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> > > Hi Matthias,
> > >
> > > Thanks for your comments. I agree that this is not the best way to do.
> A
> > > bit of history behind this design.
> > >
> > > Prior doing this, I tried to provide ProcessorContext itself as an
> > argument
> > > in Rich interfaces. However, we dont want to give users that
> flexibility
> > > and “power”. Moreover, ProcessorContext contains processor level
> > > information and not Record level info. The only thing we need ij
> > > ProcessorContext is commit() method.
> > >
> > > So, as far as I understood, we need recor context (offset, timestamp
> and
> > > etc) but also we need a commit() method ( we dont want to provide
> > > ProcessorContext as a parameter so users can use
> > ProcessorContext.commit()
> > > ).
> > >
> > > As a result, I thought to “propagate” commit() call from RecordContext
> to
> > > ProcessorContext() .
> > >
> > >
> > > If there is a misunderstanding in motvation/discussion of KIP/included
> > > jiras please let me know.
> > >
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > >
> > > On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> > >
> > >> I am personally still not convinced, that we should add `commit()` at
> > all.
> > >>
> > >> @Guozhang: you created the original Jira. Can you elaborate a little
> > >> bit? Isn't requesting commits a low level API that should not be
> exposed
> > >> in the DSL? Just want to understand the motivation better. Why would
> > >> anybody that uses the DSL ever want to request a commit? To me,
> > >> requesting commits is useful if you manipulated state explicitly, ie,
> > >> via Processor API.
> > >>
> > >> Also, for the solution: it seem rather unnatural to me, that we add
> > >> `commit()` to `RecordContext` -- from my understanding,
> `RecordContext`
> > >> is an helper object that provide access to record meta data.
> Requesting
> > >> a commit is something quite different. Additionally, a commit does not
> > >> commit a specific record but a `RecrodContext` is for a specific
> record.
> > >>
> > >> To me, this does not seem to be a sound API design if we follow this
> > path.
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >>
> > >> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> > >>> Hi,
> > >>>
> > >>> Thanks for your suggestions.
> > >>>
> > >>> I have some comments, to make sure that there is no misunderstanding.
> > >>>
> > >>>
> > >>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> > enforce
> > >>>> user to consolidate this call as
> > >>>> "processorContext.recordContext().commit()". And internal
> > implementation
> > >>>> of
> > >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> changed
> > to
> > >>>> this call.
> > >>>
> > >>>
> > >>> - I think we should not deprecate `ProcessorContext.commit()`. The
> main
> > >>> intuition that we introduce `commit()` in `RecordContext` is that,
> > >>> `RecordContext` is the one which is provided in Rich interfaces. So
> if
> > >> user
> > >>> wants to commit, then there should be some method inside
> > `RecordContext`
> > >> to
> > >>> do so. Internally, `RecordContext.commit()` calls
> > >>> `ProcessorContext.commit()`  (see the last code snippet in KIP-159):
> > >>>
> > >>> @Override
> > >>>     public void process(final K1 key, final V1 value) {
> > >>>
> > >>>         recordContext = new RecordContext() {               //
> > >>> recordContext initialization is added in this KIP
> > >>>             @Override
> > >>>             public void commit() {
> > >>>                 context().commit();
> > >>>             }
> > >>>
> > >>>             @Override
> > >>>             public long offset() {
> > >>>                 return context().recordContext().offset();
> > >>>             }
> > >>>
> > >>>             @Override
> > >>>             public long timestamp() {
> > >>>                 return context().recordContext().timestamp();
> > >>>             }
> > >>>
> > >>>             @Override
> > >>>             public String topic() {
> > >>>                 return context().recordContext().topic();
> > >>>             }
> > >>>
> > >>>             @Override
> > >>>             public int partition() {
> > >>>                 return context().recordContext().partition();
> > >>>             }
> > >>>       };
> > >>>
> > >>>
> > >>> So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.
> > >>>
> > >>>
> > >>> 2. Add the `task` reference to the impl class,
> > `ProcessorRecordContext`,
> > >> so
> > >>>> that it can implement the commit call itself.
> > >>>
> > >>>
> > >>> - Actually, I don't think that we need `commit()` in
> > >>> `ProcessorRecordContext`. 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 `RecordContext()` just
> > only
> > >> to
> > >>> 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 KIP, we made `RecordContext` to be the parent class of
> > >>> `ProcessorRecordContext`, just because of they share quite amount of
> > >>> methods and it is logical to enable inheritance between those two.
> > >>>
> > >>> 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
> snippet
> > >>>> below 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 has gone though many
> > changes
> > >>> based on user/developer needs, both in
> > >> documentation-/implementation-wise.
> > >>>
> > >>>
> > >>> Cheers,
> > >>> Jeyhun
> > >>>
> > >>>
> > >>>
> > >>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >>>
> > >>>> Thanks for the information Jeyhun. I had also forgot about
> KAFKA-3907
> > >> with
> > >>>> this 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:
> > >>>>
> > >>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> > enforce
> > >>>> user to consolidate this call as
> > >>>> "processorContext.recordContext().commit()". And internal
> > implementation
> > >>>> of
> > >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also
> changed
> > to
> > >>>> this call.
> > >>>>
> > >>>> 2. Add the `task` reference to the impl class,
> > >> `ProcessorRecordContext`, so
> > >>>> that it can implement the commit call itself.
> > >>>>
> > >>>> 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
> snippet
> > >>>> below would need to be updated as well.
> > >>>>
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>>
> > >>>>
> > >>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> > matthias@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.
> > >>>>>
> > >>>>>
> > >>>>> -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
> > >>>> 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/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. Sax <
> > >> matthias@confluent.io
> > >>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> 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 DSL users should not worry about --
> > and
> > >>>> if
> > >>>>>>> one really needs this, an advanced user can still insert a dummy
> > >>>>>>> `transform` to request a commit from there.
> > >>>>>>>
> > >>>>>>> -Matthias
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> > >>>>>>>> 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
> argument
> > to
> > >>>>> Rich
> > >>>>>>>> methods:
> > >>>>>>>>
> > >>>>>>>> instead of
> > >>>>>>>> public interface RichValueMapper<V, VR, K> {
> > >>>>>>>>     VR apply(final V value,
> > >>>>>>>>              final K key,
> > >>>>>>>>              final RecordContext recordContext);
> > >>>>>>>> }
> > >>>>>>>>
> > >>>>>>>> we can adopt
> > >>>>>>>>
> > >>>>>>>> public interface RichValueMapper<V, VR, K> {
> > >>>>>>>>     VR apply(final V value,
> > >>>>>>>>              final K key,
> > >>>>>>>>              final RecordContext recordContext,
> > >>>>>>>>              final ProcessorContext processorContext);
> > >>>>>>>> }
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> However, in this case, a user can get confused as
> ProcessorContext
> > >>>> and
> > >>>>>>>> RecordContext share some methods with the same name.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Jeyhun
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <
> wangguoz@gmail.com
> > >
> > >>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> 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 whole task up to this processed record, not only that
> > single
> > >>>>>>> record
> > >>>>>>>>> itself.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> Guozhang
> > >>>>>>>>>
> > >>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> > >>>> je.karimov@gmail.com
> > >>>>>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> 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<V1, V2, VR, K> {
> > >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> > final
> > >>>>>>>>>>> RecordContext
> > >>>>>>>>>>> recordContext);
> > >>>>>>>>>>> }
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> - Fixed.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 2. Some of the listed functions are not necessary since their
> > >>>> pairing
> > >>>>>>>>> APIs
> > >>>>>>>>>>> are being deprecated in 1.0 already:
> > >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> > >>>> super
> > >>>>> K,
> > >>>>>>>>> ?
> > >>>>>>>>>>> super V, KR> selector,
> > >>>>>>>>>>>                                    final Serde<KR> keySerde,
> > >>>>>>>>>>>                                    final Serde<V> valSerde);
> > >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >>>>>>>>>>>                                  final RichValueJoiner<?
> super
> > K,
> > >>>> ?
> > >>>>>>>>> super
> > >>>>>>>>>>> V,
> > >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> > >>>>>>>>>>>                                  final Serde<K> keySerde,
> > >>>>>>>>>>>                                  final Serde<V> valSerde);
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Fixed
> > >>>>>>>>>>
> > >>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> > combo
> > >>>> of
> > >>>>>>> both
> > >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> > >>>>>>>>> subtractor,
> > >>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> functions
> > >>>> for
> > >>>>>>>>> both;
> > >>>>>>>>>>> so that we can have less overloads and let users who only
> want
> > to
> > >>>>>>>>> access
> > >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> > >> example:
> > >>>>>>>>>>>
> > >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > >>>>>>>>> globalKTable,
> > >>>>>>>>>>>                                  final RichKeyValueMapper<?
> > super
> > >>>>> K, ?
> > >>>>>>>>>>> super
> > >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> > >>>>>>>>>>>                                  final RichValueJoiner<?
> super
> > K,
> > >>>> ?
> > >>>>>>>>> super
> > >>>>>>>>>>> V,
> > >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Agreed. Fixed.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
> its
> > >>>>>>>>>>> Initializer also "rich" functions? I.e.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> - It was a typo. Fixed.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 5. We need to move "RecordContext" from
> > o.a.k.processor.internals
> > >>>> to
> > >>>>>>>>>>> o.a.k.processor.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> > >>>>> ProcessorContext
> > >>>>>>>>> to
> > >>>>>>>>>>> RecordContext?
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -
> > >>>>>>>>>> Because it makes sense logically and  to reduce code
> maintenance
> > >>>>> (both
> > >>>>>>>>>> interfaces have offset() timestamp() topic() partition()
> > >>>> methods),  I
> > >>>>>>>>>> inherit ProcessorContext from RecordContext.
> > >>>>>>>>>> Since we need commit() method both in ProcessorContext and in
> > >>>>>>>>> RecordContext
> > >>>>>>>>>> I move commit() method to parent class (RecordContext).
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Cheers,
> > >>>>>>>>>> Jeyhun
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> > >>>> wangguoz@gmail.com>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> 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<V1, V2, VR, K> {
> > >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> > final
> > >>>>>>>>>>> RecordContext
> > >>>>>>>>>>> recordContext);
> > >>>>>>>>>>> }
> > >>>>>>>>>>>
> > >>>>>>>>>>> My motivation is that for lambda expression in J8, users that
> > >>>> would
> > >>>>>>> not
> > >>>>>>>>>>> care about the key but only the context, or vice versa, is
> > likely
> > >>>> to
> > >>>>>>>>>> write
> > >>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting
> the
> > >>>> dummy
> > >>>>>>> at
> > >>>>>>>>>> the
> > >>>>>>>>>>> beginning of the parameter list. Generally speaking we'd like
> > to
> > >>>>> make
> > >>>>>>>>> all
> > >>>>>>>>>>> the "necessary" parameters prior to optional ones.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 2. Some of the listed functions are not necessary since their
> > >>>>> pairing
> > >>>>>>>>>> APIs
> > >>>>>>>>>>> are being deprecated in 1.0 already:
> > >>>>>>>>>>>
> > >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> > >>>> super
> > >>>>> K,
> > >>>>>>>>> ?
> > >>>>>>>>>>> super V, KR> selector,
> > >>>>>>>>>>>                                    final Serde<KR> keySerde,
> > >>>>>>>>>>>                                    final Serde<V> valSerde);
> > >>>>>>>>>>>
> > >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >>>>>>>>>>>                                  final RichValueJoiner<?
> super
> > K,
> > >>>> ?
> > >>>>>>>>> super
> > >>>>>>>>>>> V,
> > >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> > >>>>>>>>>>>                                  final Serde<K> keySerde,
> > >>>>>>>>>>>                                  final Serde<V> valSerde);
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> > combo
> > >>>> of
> > >>>>>>>>> both
> > >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> > >>>>>>>>> subtractor,
> > >>>>>>>>>>> I'm wondering if we can just keep one that use "rich"
> functions
> > >>>> for
> > >>>>>>>>> both;
> > >>>>>>>>>>> so that we can have less overloads and let users who only
> want
> > to
> > >>>>>>>>> access
> > >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> > >> example:
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > >>>>>>>>> globalKTable,
> > >>>>>>>>>>>                                  final RichKeyValueMapper<?
> > super
> > >>>>> K, ?
> > >>>>>>>>>>> super
> > >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> > >>>>>>>>>>>                                  final RichValueJoiner<?
> super
> > K,
> > >>>> ?
> > >>>>>>>>> super
> > >>>>>>>>>>> V,
> > >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> > >>>>>>>>>>>
> > >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> > >>>>> initializer,
> > >>>>>>>>>>>                              final RichAggregator<? super K,
> ?
> > >>>> super
> > >>>>>>> V,
> > >>>>>>>>>> VR>
> > >>>>>>>>>>> aggregator,
> > >>>>>>>>>>>                              final Materialized<K, VR,
> > >>>>>>>>>> KeyValueStore<Bytes,
> > >>>>>>>>>>> byte[]>> materialized);
> > >>>>>>>>>>>
> > >>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
> > >> deprecated
> > >>>>> so
> > >>>>>>>>> we
> > >>>>>>>>>> do
> > >>>>>>>>>>> not need to add its rich functions any more.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
> > its
> > >>>>>>>>>>> Initializer also "rich" functions? I.e.
> > >>>>>>>>>>>
> > >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> > RichInitializer<VR,
> > >>>> K>
> > >>>>>>>>>>> initializer,
> > >>>>>>>>>>>                                        final RichAggregator<?
> > >>>> super
> > >>>>> K,
> > >>>>>>>>> ?
> > >>>>>>>>>>> super V, VR> aggregator);
> > >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> > RichInitializer<VR,
> > >>>> K>
> > >>>>>>>>>>> initializer,
> > >>>>>>>>>>>                                        final RichAggregator<?
> > >>>> super
> > >>>>> K,
> > >>>>>>>>> ?
> > >>>>>>>>>>> super V, VR> aggregator,
> > >>>>>>>>>>>                                        final Materialized<K,
> > VR,
> > >>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 5. We need to move "RecordContext" from
> > o.a.k.processor.internals
> > >>>> to
> > >>>>>>>>>>> o.a.k.processor.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> > >>>>> ProcessorContext
> > >>>>>>>>> to
> > >>>>>>>>>>> RecordContext? Conceptually I think it would better staying
> in
> > >> the
> > >>>>>>>>>>> ProcessorContext. Do you find this not doable in the internal
> > >>>>>>>>>>> implementations?
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Guozhang
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yuzhihong@gmail.com
> >
> > >>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>>    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 <
> > >>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> 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 to KIP.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > initializer,
> > >>>>>>>>>>>>>>                              final Aggregator<? super K, ?
> > >>>> super
> > >>>>>>>>> V,
> > >>>>>>>>>>> VR>
> > >>>>>>>>>>>>>> adder,
> > >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments
> in
> > >> the
> > >>>>>>>>>>> related
> > >>>>>>>>>>>>> method. So, I had to overload all possible their Rich
> > >>>>> counterparts:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
> > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > initializer,
> > >>>>>>>>>>>>>                              final Aggregator<? super K, ?
> > >> super
> > >>>>> V,
> > >>>>>>>>>> VR>
> > >>>>>>>>>>>>> adder,
> > >>>>>>>>>>>>>                              final RichAggregator<? super
> K,
> > ?
> > >>>>>>>>> super
> > >>>>>>>>>> V,
> > >>>>>>>>>>>> VR>
> > >>>>>>>>>>>>> subtractor,
> > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > >>>>>>>>>>>> KeyValueStore<Bytes,
> > >>>>>>>>>>>>> byte[]>> materialized);
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
> > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > initializer,
> > >>>>>>>>>>>>>                              final RichAggregator<? super
> K,
> > ?
> > >>>>>>>>> super
> > >>>>>>>>>> V,
> > >>>>>>>>>>>> VR>
> > >>>>>>>>>>>>> adder,
> > >>>>>>>>>>>>>                              final Aggregator<? super K, ?
> > >> super
> > >>>>> V,
> > >>>>>>>>>> VR>
> > >>>>>>>>>>>>> subtractor,
> > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > >>>>>>>>>>>> KeyValueStore<Bytes,
> > >>>>>>>>>>>>> byte[]>> materialized);
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // both adder and subtractor are rich
> > >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > initializer,
> > >>>>>>>>>>>>>                              final RichAggregator<? super
> K,
> > ?
> > >>>>>>>>> super
> > >>>>>>>>>> V,
> > >>>>>>>>>>>> VR>
> > >>>>>>>>>>>>> adder,
> > >>>>>>>>>>>>>                              final RichAggregator<? super
> K,
> > ?
> > >>>>>>>>> super
> > >>>>>>>>>> V,
> > >>>>>>>>>>>> VR>
> > >>>>>>>>>>>>> subtractor,
> > >>>>>>>>>>>>>                              final Materialized<K, VR,
> > >>>>>>>>>>>> KeyValueStore<Bytes,
> > >>>>>>>>>>>>> byte[]>> materialized);
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> > >>>>>>>>>>>>>>    void commit () {
> > >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is
> > not
> > >>>>>>>>>>>> supported
> > >>>>>>>>>>>>> in
> > >>>>>>>>>>>>>> this context");
> > >>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> the
> > PR
> > >>>> ?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> - I added some comments both inside and outside the code
> > >>>> snippets
> > >>>>>>>>> in
> > >>>>>>>>>>> KIP.
> > >>>>>>>>>>>>> Specifically, for the code snippet above, we add *commit()*
> > >>>> method
> > >>>>>>>>> to
> > >>>>>>>>>>>>> *RecordContext* interface.
> > >>>>>>>>>>>>> However, we want  *commit()* method to be used only for
> > >>>>>>>>>> *RecordContext*
> > >>>>>>>>>>>>> instances (at least for now), so we add
> > >>>>>>>>> UnsupportedOperationException
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>> all classes/interfaces that extend/implement
> *RecordContext.*
> > >>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
> > within
> > >>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
> > >>>> required
> > >>>>>>>>>>>>> Processors and 3) pass it as an argument to the related
> Rich
> > >>>>>>>>>> interfaces
> > >>>>>>>>>>>>> inside Processors.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <
> yuzhihong@gmail.com>
> > >>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 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.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> > >> initializer,
> > >>>>>>>>>>>>>>                              final Aggregator<? super K, ?
> > >>>> super
> > >>>>>>>>> V,
> > >>>>>>>>>>> VR>
> > >>>>>>>>>>>>>> adder,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>   public RecordContext recordContext() {
> > >>>>>>>>>>>>>>     return this.recordContext();
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>    void commit () {
> > >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is
> > not
> > >>>>>>>>>>>> supported
> > >>>>>>>>>>>>> in
> > >>>>>>>>>>>>>> this context");
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Is the exception going to be replaced with real code in
> the
> > PR
> > >>>> ?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Cheers
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> > >>>>>>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>> damian.guy@gmail.com
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>>> 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 Jeyhun Karimov <
> > >>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as
> KIP-159
> > is
> > >>>>>>>>>> not
> > >>>>>>>>>>>>> voted
> > >>>>>>>>>>>>>>> yet
> > >>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> > >>>>>>>>>> "atomic"
> > >>>>>>>>>>>>>> merge).
> > >>>>>>>>>>>>>>>>> So
> > >>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> > >>>>>>>>> some
> > >>>>>>>>>>>> minor
> > >>>>>>>>>>>>>>>>> updates
> > >>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct
> me
> > >>>>>>>>> if
> > >>>>>>>>>> I
> > >>>>>>>>>>> am
> > >>>>>>>>>>>>>> wrong
> > >>>>>>>>>>>>>>>>> or I
> > >>>>>>>>>>>>>>>>>> misunderstood.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > >>>>>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> +1
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > >>>>>>>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>> would
> > >>>>>>>>>>>>> still
> > >>>>>>>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and vote
> > >>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > >>>>>>>>>>>> yuzhihong@gmail.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> +1
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > >>>>>>>>>>>>>> bbejeck@gmail.com
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> > >>>>>>>>>>>> ValueXXXXWithKey
> > >>>>>>>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>>>>>>>> into 1
> > >>>>>>>>>>>>>>>>>>>>>> interface.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>> Bill
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> > >>>>>>>>>>>>>> implements
> > >>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> > >>>>>>>>>>>>>>>>>>>>>>> ......
> > >>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> > >>>>>>>>>>> RichXXX
> > >>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>> XXXWithKey
> > >>>>>>>>>>>>>>>>>>>>>> inside
> > >>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> > >>>>>>>>> to
> > >>>>>>>>>>>> revert
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>> design
> > >>>>>>>>>>>>>>>>>>>>>>> to this again.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > >>>>>>>>>>>>>>>>> damian.guy@gmail.com
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> > >>>>>>>>> was
> > >>>>>>>>>>>>>> thinking
> > >>>>>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> > >>>>>>>>> RichXXXX
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
> > >>>>>>>>>>>>>>>>>>>>>> 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?
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > >>>>>>>>>>> confluence/pages/viewpage.
> > >>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> > >>>>>>>>>> Karimov <
> > >>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> > >>>>>>>>>> issue.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> > >>>>>>>>>> Guy
> > >>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>>>>> overrides.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> > >>>>>>>>>> RecordContext,
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>> right.
> > >>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> > >>>>>>>>> the
> > >>>>>>>>>>>> full
> > >>>>>>>>>>>>>>>>> picture
> > >>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>> parts
> > >>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> > >>>>>>>>> as
> > >>>>>>>>>> I
> > >>>>>>>>>>>>>> thought.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> > >>>>>>>>> Damian
> > >>>>>>>>>>> Guy
> > >>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>>>>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
Hi,

I removed the 'commit()' feature, as we discussed. It simplified  the
overall design of KIP a lot.
If it is ok, I would like to start a VOTE thread.

Cheers,
Jeyhun

On Fri, Oct 27, 2017 at 5:28 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Thanks. I understand what you are saying, but I don't agree that
>
> > but also we need a commit() method
>
> I would just not provide `commit()` at DSL level and close the
> corresponding Jira as "not a problem" or similar.
>
>
> -Matthias
>
> On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> > Hi Matthias,
> >
> > Thanks for your comments. I agree that this is not the best way to do. A
> > bit of history behind this design.
> >
> > Prior doing this, I tried to provide ProcessorContext itself as an
> argument
> > in Rich interfaces. However, we dont want to give users that flexibility
> > and “power”. Moreover, ProcessorContext contains processor level
> > information and not Record level info. The only thing we need ij
> > ProcessorContext is commit() method.
> >
> > So, as far as I understood, we need recor context (offset, timestamp and
> > etc) but also we need a commit() method ( we dont want to provide
> > ProcessorContext as a parameter so users can use
> ProcessorContext.commit()
> > ).
> >
> > As a result, I thought to “propagate” commit() call from RecordContext to
> > ProcessorContext() .
> >
> >
> > If there is a misunderstanding in motvation/discussion of KIP/included
> > jiras please let me know.
> >
> >
> > Cheers,
> > Jeyhun
> >
> >
> > On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <ma...@confluent.io>
> wrote:
> >
> >> I am personally still not convinced, that we should add `commit()` at
> all.
> >>
> >> @Guozhang: you created the original Jira. Can you elaborate a little
> >> bit? Isn't requesting commits a low level API that should not be exposed
> >> in the DSL? Just want to understand the motivation better. Why would
> >> anybody that uses the DSL ever want to request a commit? To me,
> >> requesting commits is useful if you manipulated state explicitly, ie,
> >> via Processor API.
> >>
> >> Also, for the solution: it seem rather unnatural to me, that we add
> >> `commit()` to `RecordContext` -- from my understanding, `RecordContext`
> >> is an helper object that provide access to record meta data. Requesting
> >> a commit is something quite different. Additionally, a commit does not
> >> commit a specific record but a `RecrodContext` is for a specific record.
> >>
> >> To me, this does not seem to be a sound API design if we follow this
> path.
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> >>> Hi,
> >>>
> >>> Thanks for your suggestions.
> >>>
> >>> I have some comments, to make sure that there is no misunderstanding.
> >>>
> >>>
> >>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> enforce
> >>>> user to consolidate this call as
> >>>> "processorContext.recordContext().commit()". And internal
> implementation
> >>>> of
> >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed
> to
> >>>> this call.
> >>>
> >>>
> >>> - I think we should not deprecate `ProcessorContext.commit()`. The main
> >>> intuition that we introduce `commit()` in `RecordContext` is that,
> >>> `RecordContext` is the one which is provided in Rich interfaces. So if
> >> user
> >>> wants to commit, then there should be some method inside
> `RecordContext`
> >> to
> >>> do so. Internally, `RecordContext.commit()` calls
> >>> `ProcessorContext.commit()`  (see the last code snippet in KIP-159):
> >>>
> >>> @Override
> >>>     public void process(final K1 key, final V1 value) {
> >>>
> >>>         recordContext = new RecordContext() {               //
> >>> recordContext initialization is added in this KIP
> >>>             @Override
> >>>             public void commit() {
> >>>                 context().commit();
> >>>             }
> >>>
> >>>             @Override
> >>>             public long offset() {
> >>>                 return context().recordContext().offset();
> >>>             }
> >>>
> >>>             @Override
> >>>             public long timestamp() {
> >>>                 return context().recordContext().timestamp();
> >>>             }
> >>>
> >>>             @Override
> >>>             public String topic() {
> >>>                 return context().recordContext().topic();
> >>>             }
> >>>
> >>>             @Override
> >>>             public int partition() {
> >>>                 return context().recordContext().partition();
> >>>             }
> >>>       };
> >>>
> >>>
> >>> So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.
> >>>
> >>>
> >>> 2. Add the `task` reference to the impl class,
> `ProcessorRecordContext`,
> >> so
> >>>> that it can implement the commit call itself.
> >>>
> >>>
> >>> - Actually, I don't think that we need `commit()` in
> >>> `ProcessorRecordContext`. 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 `RecordContext()` just
> only
> >> to
> >>> 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 KIP, we made `RecordContext` to be the parent class of
> >>> `ProcessorRecordContext`, just because of they share quite amount of
> >>> methods and it is logical to enable inheritance between those two.
> >>>
> >>> 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 snippet
> >>>> below 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 has gone though many
> changes
> >>> based on user/developer needs, both in
> >> documentation-/implementation-wise.
> >>>
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>>
> >>>
> >>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> Thanks for the information Jeyhun. I had also forgot about KAFKA-3907
> >> with
> >>>> this 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:
> >>>>
> >>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to
> enforce
> >>>> user to consolidate this call as
> >>>> "processorContext.recordContext().commit()". And internal
> implementation
> >>>> of
> >>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed
> to
> >>>> this call.
> >>>>
> >>>> 2. Add the `task` reference to the impl class,
> >> `ProcessorRecordContext`, so
> >>>> that it can implement the commit call itself.
> >>>>
> >>>> 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 snippet
> >>>> below would need to be updated as well.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>>
> >>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <
> matthias@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.
> >>>>>
> >>>>>
> >>>>> -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
> >>>> 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/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. Sax <
> >> matthias@confluent.io
> >>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> 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 DSL users should not worry about --
> and
> >>>> if
> >>>>>>> one really needs this, an advanced user can still insert a dummy
> >>>>>>> `transform` to request a commit from there.
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>>
> >>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>>>>>>> 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 argument
> to
> >>>>> Rich
> >>>>>>>> methods:
> >>>>>>>>
> >>>>>>>> instead of
> >>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>     VR apply(final V value,
> >>>>>>>>              final K key,
> >>>>>>>>              final RecordContext recordContext);
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>> we can adopt
> >>>>>>>>
> >>>>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>>>     VR apply(final V value,
> >>>>>>>>              final K key,
> >>>>>>>>              final RecordContext recordContext,
> >>>>>>>>              final ProcessorContext processorContext);
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> However, in this case, a user can get confused as ProcessorContext
> >>>> and
> >>>>>>>> RecordContext share some methods with the same name.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Jeyhun
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wangguoz@gmail.com
> >
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> 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 whole task up to this processed record, not only that
> single
> >>>>>>> record
> >>>>>>>>> itself.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> >>>> je.karimov@gmail.com
> >>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> final
> >>>>>>>>>>> RecordContext
> >>>>>>>>>>> recordContext);
> >>>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> - Fixed.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 2. Some of the listed functions are not necessary since their
> >>>> pairing
> >>>>>>>>> APIs
> >>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> >>>> super
> >>>>> K,
> >>>>>>>>> ?
> >>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>                                    final Serde<KR> keySerde,
> >>>>>>>>>>>                                    final Serde<V> valSerde);
> >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>>>                                  final RichValueJoiner<? super
> K,
> >>>> ?
> >>>>>>>>> super
> >>>>>>>>>>> V,
> >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Fixed
> >>>>>>>>>>
> >>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> combo
> >>>> of
> >>>>>>> both
> >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>>>>>>> subtractor,
> >>>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
> >>>> for
> >>>>>>>>> both;
> >>>>>>>>>>> so that we can have less overloads and let users who only want
> to
> >>>>>>>>> access
> >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> >> example:
> >>>>>>>>>>>
> >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>>>> globalKTable,
> >>>>>>>>>>>                                  final RichKeyValueMapper<?
> super
> >>>>> K, ?
> >>>>>>>>>>> super
> >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>>>                                  final RichValueJoiner<? super
> K,
> >>>> ?
> >>>>>>>>> super
> >>>>>>>>>>> V,
> >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Agreed. Fixed.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> - It was a typo. Fixed.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 5. We need to move "RecordContext" from
> o.a.k.processor.internals
> >>>> to
> >>>>>>>>>>> o.a.k.processor.
> >>>>>>>>>>>
> >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>>>> ProcessorContext
> >>>>>>>>> to
> >>>>>>>>>>> RecordContext?
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -
> >>>>>>>>>> Because it makes sense logically and  to reduce code maintenance
> >>>>> (both
> >>>>>>>>>> interfaces have offset() timestamp() topic() partition()
> >>>> methods),  I
> >>>>>>>>>> inherit ProcessorContext from RecordContext.
> >>>>>>>>>> Since we need commit() method both in ProcessorContext and in
> >>>>>>>>> RecordContext
> >>>>>>>>>> I move commit() method to parent class (RecordContext).
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Jeyhun
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> >>>> wangguoz@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key,
> final
> >>>>>>>>>>> RecordContext
> >>>>>>>>>>> recordContext);
> >>>>>>>>>>> }
> >>>>>>>>>>>
> >>>>>>>>>>> My motivation is that for lambda expression in J8, users that
> >>>> would
> >>>>>>> not
> >>>>>>>>>>> care about the key but only the context, or vice versa, is
> likely
> >>>> to
> >>>>>>>>>> write
> >>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting the
> >>>> dummy
> >>>>>>> at
> >>>>>>>>>> the
> >>>>>>>>>>> beginning of the parameter list. Generally speaking we'd like
> to
> >>>>> make
> >>>>>>>>> all
> >>>>>>>>>>> the "necessary" parameters prior to optional ones.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 2. Some of the listed functions are not necessary since their
> >>>>> pairing
> >>>>>>>>>> APIs
> >>>>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>>>
> >>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> >>>> super
> >>>>> K,
> >>>>>>>>> ?
> >>>>>>>>>>> super V, KR> selector,
> >>>>>>>>>>>                                    final Serde<KR> keySerde,
> >>>>>>>>>>>                                    final Serde<V> valSerde);
> >>>>>>>>>>>
> >>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>>>                                  final RichValueJoiner<? super
> K,
> >>>> ?
> >>>>>>>>> super
> >>>>>>>>>>> V,
> >>>>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 3. For a few functions where we are adding three APIs for a
> combo
> >>>> of
> >>>>>>>>> both
> >>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>>>>>>> subtractor,
> >>>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
> >>>> for
> >>>>>>>>> both;
> >>>>>>>>>>> so that we can have less overloads and let users who only want
> to
> >>>>>>>>> access
> >>>>>>>>>>> one of them to just use dummy parameter declarations. For
> >> example:
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>>>> globalKTable,
> >>>>>>>>>>>                                  final RichKeyValueMapper<?
> super
> >>>>> K, ?
> >>>>>>>>>>> super
> >>>>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>>>                                  final RichValueJoiner<? super
> K,
> >>>> ?
> >>>>>>>>> super
> >>>>>>>>>>> V,
> >>>>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>>>
> >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> >>>>> initializer,
> >>>>>>>>>>>                              final RichAggregator<? super K, ?
> >>>> super
> >>>>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> aggregator,
> >>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>
> >>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
> >> deprecated
> >>>>> so
> >>>>>>>>> we
> >>>>>>>>>> do
> >>>>>>>>>>> not need to add its rich functions any more.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make
> its
> >>>>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>>>
> >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> RichInitializer<VR,
> >>>> K>
> >>>>>>>>>>> initializer,
> >>>>>>>>>>>                                        final RichAggregator<?
> >>>> super
> >>>>> K,
> >>>>>>>>> ?
> >>>>>>>>>>> super V, VR> aggregator);
> >>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final
> RichInitializer<VR,
> >>>> K>
> >>>>>>>>>>> initializer,
> >>>>>>>>>>>                                        final RichAggregator<?
> >>>> super
> >>>>> K,
> >>>>>>>>> ?
> >>>>>>>>>>> super V, VR> aggregator,
> >>>>>>>>>>>                                        final Materialized<K,
> VR,
> >>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 5. We need to move "RecordContext" from
> o.a.k.processor.internals
> >>>> to
> >>>>>>>>>>> o.a.k.processor.
> >>>>>>>>>>>
> >>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>>>> ProcessorContext
> >>>>>>>>> to
> >>>>>>>>>>> RecordContext? Conceptually I think it would better staying in
> >> the
> >>>>>>>>>>> ProcessorContext. Do you find this not doable in the internal
> >>>>>>>>>>> implementations?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Guozhang
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
> >>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>>    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 <
> >>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 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 to KIP.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> initializer,
> >>>>>>>>>>>>>>                              final Aggregator<? super K, ?
> >>>> super
> >>>>>>>>> V,
> >>>>>>>>>>> VR>
> >>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in
> >> the
> >>>>>>>>>>> related
> >>>>>>>>>>>>> method. So, I had to overload all possible their Rich
> >>>>> counterparts:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
> >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> initializer,
> >>>>>>>>>>>>>                              final Aggregator<? super K, ?
> >> super
> >>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>                              final RichAggregator<? super K,
> ?
> >>>>>>>>> super
> >>>>>>>>>> V,
> >>>>>>>>>>>> VR>
> >>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
> >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> initializer,
> >>>>>>>>>>>>>                              final RichAggregator<? super K,
> ?
> >>>>>>>>> super
> >>>>>>>>>> V,
> >>>>>>>>>>>> VR>
> >>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>                              final Aggregator<? super K, ?
> >> super
> >>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // both adder and subtractor are rich
> >>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> initializer,
> >>>>>>>>>>>>>                              final RichAggregator<? super K,
> ?
> >>>>>>>>> super
> >>>>>>>>>> V,
> >>>>>>>>>>>> VR>
> >>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>                              final RichAggregator<? super K,
> ?
> >>>>>>>>> super
> >>>>>>>>>> V,
> >>>>>>>>>>>> VR>
> >>>>>>>>>>>>> subtractor,
> >>>>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is
> not
> >>>>>>>>>>>> supported
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>> this context");
> >>>>>>>>>>>>>> Is the exception going to be replaced with real code in the
> PR
> >>>> ?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - I added some comments both inside and outside the code
> >>>> snippets
> >>>>>>>>> in
> >>>>>>>>>>> KIP.
> >>>>>>>>>>>>> Specifically, for the code snippet above, we add *commit()*
> >>>> method
> >>>>>>>>> to
> >>>>>>>>>>>>> *RecordContext* interface.
> >>>>>>>>>>>>> However, we want  *commit()* method to be used only for
> >>>>>>>>>> *RecordContext*
> >>>>>>>>>>>>> instances (at least for now), so we add
> >>>>>>>>> UnsupportedOperationException
> >>>>>>>>>>> in
> >>>>>>>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
> >>>>>>>>>>>>> In general, 1) we make RecordContext publicly available
> within
> >>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
> >>>> required
> >>>>>>>>>>>>> Processors and 3) pass it as an argument to the related Rich
> >>>>>>>>>> interfaces
> >>>>>>>>>>>>> inside Processors.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> >> initializer,
> >>>>>>>>>>>>>>                              final Aggregator<? super K, ?
> >>>> super
> >>>>>>>>> V,
> >>>>>>>>>>> VR>
> >>>>>>>>>>>>>> adder,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>   public RecordContext recordContext() {
> >>>>>>>>>>>>>>     return this.recordContext();
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is
> not
> >>>>>>>>>>>> supported
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>> this context");
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Is the exception going to be replaced with real code in the
> PR
> >>>> ?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Cheers
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> >>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>> 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 Jeyhun Karimov <
> >>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159
> is
> >>>>>>>>>> not
> >>>>>>>>>>>>> voted
> >>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> >>>>>>>>>> "atomic"
> >>>>>>>>>>>>>> merge).
> >>>>>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> >>>>>>>>> some
> >>>>>>>>>>>> minor
> >>>>>>>>>>>>>>>>> updates
> >>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
> >>>>>>>>> if
> >>>>>>>>>> I
> >>>>>>>>>>> am
> >>>>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>>>> or I
> >>>>>>>>>>>>>>>>>> misunderstood.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> >>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> >>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>> 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
> >>>>>>>>>> would
> >>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and vote
> >>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> >>>>>>>>>>>> yuzhihong@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> >>>>>>>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> >>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>>> interfaces
> >>>>>>>>>>>>>>>>>>>>> into 1
> >>>>>>>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> >>>>>>>>>>>>>> implements
> >>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> >>>>>>>>>>> RichXXX
> >>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>> XXXWithKey
> >>>>>>>>>>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> >>>>>>>>> to
> >>>>>>>>>>>> revert
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>>>> to this again.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >>>>>>>>>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> >>>>>>>>> was
> >>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> >>>>>>>>> RichXXXX
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>>>>>>> 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?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>>>>>>>>> confluence/pages/viewpage.
> >>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> >>>>>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> >>>>>>>>>> issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> >>>>>>>>>> Guy
> >>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>> overrides.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> >>>>>>>>>> RecordContext,
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> right.
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> >>>>>>>>> the
> >>>>>>>>>>>> full
> >>>>>>>>>>>>>>>>> picture
> >>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>> parts
> >>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> >>>>>>>>> as
> >>>>>>>>>> I
> >>>>>>>>>>>>>> thought.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> >>>>>>>>> Damian
> >>>>>>>>>>> Guy
> >>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
> >>>>>>>>>> to
> >>>>>>>>>>>>> cast a
> >>>>>>>>>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
> >>>>>>>>>>>>>>>>>>> InternalProcessorContext
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> init()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
> >>>>>>>>>> and i
> >>>>>>>>>>>>> think
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>> remain
> >>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
> >>>>>>>>>>>>> recordContext()
> >>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
> >>>>>>>>>>>>> ProcessorContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
> >>>>>>>>>> showing:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
> >>>>>>>>>>>>>>> processorContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
> >>>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
> >>>>>>>>>> as
> >>>>>>>>>>>>> above
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sense.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> >>>>>>>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks. I understand what you are saying, but I don't agree that

> but also we need a commit() method

I would just not provide `commit()` at DSL level and close the
corresponding Jira as "not a problem" or similar.


-Matthias

On 10/27/17 3:42 PM, Jeyhun Karimov wrote:
> Hi Matthias,
> 
> Thanks for your comments. I agree that this is not the best way to do. A
> bit of history behind this design.
> 
> Prior doing this, I tried to provide ProcessorContext itself as an argument
> in Rich interfaces. However, we dont want to give users that flexibility
> and “power”. Moreover, ProcessorContext contains processor level
> information and not Record level info. The only thing we need ij
> ProcessorContext is commit() method.
> 
> So, as far as I understood, we need recor context (offset, timestamp and
> etc) but also we need a commit() method ( we dont want to provide
> ProcessorContext as a parameter so users can use ProcessorContext.commit()
> ).
> 
> As a result, I thought to “propagate” commit() call from RecordContext to
> ProcessorContext() .
> 
> 
> If there is a misunderstanding in motvation/discussion of KIP/included
> jiras please let me know.
> 
> 
> Cheers,
> Jeyhun
> 
> 
> On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <ma...@confluent.io> wrote:
> 
>> I am personally still not convinced, that we should add `commit()` at all.
>>
>> @Guozhang: you created the original Jira. Can you elaborate a little
>> bit? Isn't requesting commits a low level API that should not be exposed
>> in the DSL? Just want to understand the motivation better. Why would
>> anybody that uses the DSL ever want to request a commit? To me,
>> requesting commits is useful if you manipulated state explicitly, ie,
>> via Processor API.
>>
>> Also, for the solution: it seem rather unnatural to me, that we add
>> `commit()` to `RecordContext` -- from my understanding, `RecordContext`
>> is an helper object that provide access to record meta data. Requesting
>> a commit is something quite different. Additionally, a commit does not
>> commit a specific record but a `RecrodContext` is for a specific record.
>>
>> To me, this does not seem to be a sound API design if we follow this path.
>>
>>
>> -Matthias
>>
>>
>>
>> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
>>> Hi,
>>>
>>> Thanks for your suggestions.
>>>
>>> I have some comments, to make sure that there is no misunderstanding.
>>>
>>>
>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
>>>> user to consolidate this call as
>>>> "processorContext.recordContext().commit()". And internal implementation
>>>> of
>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
>>>> this call.
>>>
>>>
>>> - I think we should not deprecate `ProcessorContext.commit()`. The main
>>> intuition that we introduce `commit()` in `RecordContext` is that,
>>> `RecordContext` is the one which is provided in Rich interfaces. So if
>> user
>>> wants to commit, then there should be some method inside `RecordContext`
>> to
>>> do so. Internally, `RecordContext.commit()` calls
>>> `ProcessorContext.commit()`  (see the last code snippet in KIP-159):
>>>
>>> @Override
>>>     public void process(final K1 key, final V1 value) {
>>>
>>>         recordContext = new RecordContext() {               //
>>> recordContext initialization is added in this KIP
>>>             @Override
>>>             public void commit() {
>>>                 context().commit();
>>>             }
>>>
>>>             @Override
>>>             public long offset() {
>>>                 return context().recordContext().offset();
>>>             }
>>>
>>>             @Override
>>>             public long timestamp() {
>>>                 return context().recordContext().timestamp();
>>>             }
>>>
>>>             @Override
>>>             public String topic() {
>>>                 return context().recordContext().topic();
>>>             }
>>>
>>>             @Override
>>>             public int partition() {
>>>                 return context().recordContext().partition();
>>>             }
>>>       };
>>>
>>>
>>> So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.
>>>
>>>
>>> 2. Add the `task` reference to the impl class, `ProcessorRecordContext`,
>> so
>>>> that it can implement the commit call itself.
>>>
>>>
>>> - Actually, I don't think that we need `commit()` in
>>> `ProcessorRecordContext`. 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 `RecordContext()` just only
>> to
>>> 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 KIP, we made `RecordContext` to be the parent class of
>>> `ProcessorRecordContext`, just because of they share quite amount of
>>> methods and it is logical to enable inheritance between those two.
>>>
>>> 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 snippet
>>>> below 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 has gone though many changes
>>> based on user/developer needs, both in
>> documentation-/implementation-wise.
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>>
>>>
>>> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>
>>>> Thanks for the information Jeyhun. I had also forgot about KAFKA-3907
>> with
>>>> this 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:
>>>>
>>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
>>>> user to consolidate this call as
>>>> "processorContext.recordContext().commit()". And internal implementation
>>>> of
>>>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
>>>> this call.
>>>>
>>>> 2. Add the `task` reference to the impl class,
>> `ProcessorRecordContext`, so
>>>> that it can implement the commit call itself.
>>>>
>>>> 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 snippet
>>>> below would need to be updated as well.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <matthias@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.
>>>>>
>>>>>
>>>>> -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
>>>> 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/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. Sax <
>> matthias@confluent.io
>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>> 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 DSL users should not worry about -- and
>>>> if
>>>>>>> one really needs this, an advanced user can still insert a dummy
>>>>>>> `transform` to request a commit from there.
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>>>> 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 argument to
>>>>> Rich
>>>>>>>> methods:
>>>>>>>>
>>>>>>>> instead of
>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>     VR apply(final V value,
>>>>>>>>              final K key,
>>>>>>>>              final RecordContext recordContext);
>>>>>>>> }
>>>>>>>>
>>>>>>>> we can adopt
>>>>>>>>
>>>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>>>     VR apply(final V value,
>>>>>>>>              final K key,
>>>>>>>>              final RecordContext recordContext,
>>>>>>>>              final ProcessorContext processorContext);
>>>>>>>> }
>>>>>>>>
>>>>>>>>
>>>>>>>> However, in this case, a user can get confused as ProcessorContext
>>>> and
>>>>>>>> RecordContext share some methods with the same name.
>>>>>>>>
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>>
>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> 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 whole task up to this processed record, not only that single
>>>>>>> record
>>>>>>>>> itself.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>>>> je.karimov@gmail.com
>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>>>>>>> RecordContext
>>>>>>>>>>> recordContext);
>>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> - Fixed.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2. Some of the listed functions are not necessary since their
>>>> pairing
>>>>>>>>> APIs
>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
>>>> super
>>>>> K,
>>>>>>>>> ?
>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>                                    final Serde<KR> keySerde,
>>>>>>>>>>>                                    final Serde<V> valSerde);
>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>                                  final RichValueJoiner<? super K,
>>>> ?
>>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Fixed
>>>>>>>>>>
>>>>>>>>>> 3. For a few functions where we are adding three APIs for a combo
>>>> of
>>>>>>> both
>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>>>>>>> subtractor,
>>>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
>>>> for
>>>>>>>>> both;
>>>>>>>>>>> so that we can have less overloads and let users who only want to
>>>>>>>>> access
>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>> example:
>>>>>>>>>>>
>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>> globalKTable,
>>>>>>>>>>>                                  final RichKeyValueMapper<? super
>>>>> K, ?
>>>>>>>>>>> super
>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>                                  final RichValueJoiner<? super K,
>>>> ?
>>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Agreed. Fixed.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> - It was a typo. Fixed.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
>>>> to
>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>
>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>> ProcessorContext
>>>>>>>>> to
>>>>>>>>>>> RecordContext?
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -
>>>>>>>>>> Because it makes sense logically and  to reduce code maintenance
>>>>> (both
>>>>>>>>>> interfaces have offset() timestamp() topic() partition()
>>>> methods),  I
>>>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>>>> Since we need commit() method both in ProcessorContext and in
>>>>>>>>> RecordContext
>>>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>>>> wangguoz@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>>>>>>> RecordContext
>>>>>>>>>>> recordContext);
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> My motivation is that for lambda expression in J8, users that
>>>> would
>>>>>>> not
>>>>>>>>>>> care about the key but only the context, or vice versa, is likely
>>>> to
>>>>>>>>>> write
>>>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting the
>>>> dummy
>>>>>>> at
>>>>>>>>>> the
>>>>>>>>>>> beginning of the parameter list. Generally speaking we'd like to
>>>>> make
>>>>>>>>> all
>>>>>>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 2. Some of the listed functions are not necessary since their
>>>>> pairing
>>>>>>>>>> APIs
>>>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>>>
>>>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
>>>> super
>>>>> K,
>>>>>>>>> ?
>>>>>>>>>>> super V, KR> selector,
>>>>>>>>>>>                                    final Serde<KR> keySerde,
>>>>>>>>>>>                                    final Serde<V> valSerde);
>>>>>>>>>>>
>>>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>>>                                  final RichValueJoiner<? super K,
>>>> ?
>>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 3. For a few functions where we are adding three APIs for a combo
>>>> of
>>>>>>>>> both
>>>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>>>>>>> subtractor,
>>>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
>>>> for
>>>>>>>>> both;
>>>>>>>>>>> so that we can have less overloads and let users who only want to
>>>>>>>>> access
>>>>>>>>>>> one of them to just use dummy parameter declarations. For
>> example:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>>>> globalKTable,
>>>>>>>>>>>                                  final RichKeyValueMapper<? super
>>>>> K, ?
>>>>>>>>>>> super
>>>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>>>                                  final RichValueJoiner<? super K,
>>>> ?
>>>>>>>>> super
>>>>>>>>>>> V,
>>>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>>>
>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
>>>>> initializer,
>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>> super
>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>> aggregator,
>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>
>>>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
>> deprecated
>>>>> so
>>>>>>>>> we
>>>>>>>>>> do
>>>>>>>>>>> not need to add its rich functions any more.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>>>
>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
>>>> K>
>>>>>>>>>>> initializer,
>>>>>>>>>>>                                        final RichAggregator<?
>>>> super
>>>>> K,
>>>>>>>>> ?
>>>>>>>>>>> super V, VR> aggregator);
>>>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
>>>> K>
>>>>>>>>>>> initializer,
>>>>>>>>>>>                                        final RichAggregator<?
>>>> super
>>>>> K,
>>>>>>>>> ?
>>>>>>>>>>> super V, VR> aggregator,
>>>>>>>>>>>                                        final Materialized<K, VR,
>>>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
>>>> to
>>>>>>>>>>> o.a.k.processor.
>>>>>>>>>>>
>>>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>>>> ProcessorContext
>>>>>>>>> to
>>>>>>>>>>> RecordContext? Conceptually I think it would better staying in
>> the
>>>>>>>>>>> ProcessorContext. Do you find this not doable in the internal
>>>>>>>>>>> implementations?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>    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 <
>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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 to KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>>>                              final Aggregator<? super K, ?
>>>> super
>>>>>>>>> V,
>>>>>>>>>>> VR>
>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in
>> the
>>>>>>>>>>> related
>>>>>>>>>>>>> method. So, I had to overload all possible their Rich
>>>>> counterparts:
>>>>>>>>>>>>>
>>>>>>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>>                              final Aggregator<? super K, ?
>> super
>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>>>> super
>>>>>>>>>> V,
>>>>>>>>>>>> VR>
>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>
>>>>>>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>>>> super
>>>>>>>>>> V,
>>>>>>>>>>>> VR>
>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>                              final Aggregator<? super K, ?
>> super
>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>
>>>>>>>>>>>>> // both adder and subtractor are rich
>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>>>> super
>>>>>>>>>> V,
>>>>>>>>>>>> VR>
>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>>>> super
>>>>>>>>>> V,
>>>>>>>>>>>> VR>
>>>>>>>>>>>>> subtractor,
>>>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>>>>>>> supported
>>>>>>>>>>>>> in
>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
>>>> ?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> - I added some comments both inside and outside the code
>>>> snippets
>>>>>>>>> in
>>>>>>>>>>> KIP.
>>>>>>>>>>>>> Specifically, for the code snippet above, we add *commit()*
>>>> method
>>>>>>>>> to
>>>>>>>>>>>>> *RecordContext* interface.
>>>>>>>>>>>>> However, we want  *commit()* method to be used only for
>>>>>>>>>> *RecordContext*
>>>>>>>>>>>>> instances (at least for now), so we add
>>>>>>>>> UnsupportedOperationException
>>>>>>>>>>> in
>>>>>>>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
>>>>>>>>>>>>> In general, 1) we make RecordContext publicly available within
>>>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
>>>> required
>>>>>>>>>>>>> Processors and 3) pass it as an argument to the related Rich
>>>>>>>>>> interfaces
>>>>>>>>>>>>> inside Processors.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
>> initializer,
>>>>>>>>>>>>>>                              final Aggregator<? super K, ?
>>>> super
>>>>>>>>> V,
>>>>>>>>>>> VR>
>>>>>>>>>>>>>> adder,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   public RecordContext recordContext() {
>>>>>>>>>>>>>>     return this.recordContext();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>>>>>>> supported
>>>>>>>>>>>>> in
>>>>>>>>>>>>>> this context");
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
>>>> ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cheers
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 Jeyhun Karimov <
>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
>>>>>>>>>> not
>>>>>>>>>>>>> voted
>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
>>>>>>>>>> "atomic"
>>>>>>>>>>>>>> merge).
>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
>>>>>>>>> some
>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
>>>>>>>>> if
>>>>>>>>>> I
>>>>>>>>>>> am
>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>> or I
>>>>>>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>> 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
>>>>>>>>>> would
>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>> update your proposal for people to comment and vote
>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>>>>>>> yuzhihong@gmail.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>>>>>>> RichXXX
>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>>>>>>> to
>>>>>>>>>>>> revert
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>>>>>>> was
>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>>>>>>> RichXXXX
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
>>>>>>>>>> issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
>>>>>>>>>> Guy
>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>> overrides.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
>>>>>>>>>> RecordContext,
>>>>>>>>>>>> you
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> right.
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
>>>>>>>>> the
>>>>>>>>>>>> full
>>>>>>>>>>>>>>>>> picture
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>> parts
>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
>>>>>>>>> as
>>>>>>>>>> I
>>>>>>>>>>>>>> thought.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
>>>>>>>>> Damian
>>>>>>>>>>> Guy
>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
>>>>>>>>>> to
>>>>>>>>>>>>> cast a
>>>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
>>>>>>>>>>>>>>>>>>> InternalProcessorContext
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> init()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
>>>>>>>>>> and i
>>>>>>>>>>>>> think
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>> remain
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
>>>>>>>>>>>>> recordContext()
>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
>>>>>>>>>>>>> ProcessorContext.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
>>>>>>>>>> showing:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
>>>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
>>>>>>>>>> VR>
>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
>>>>>>>>>> as
>>>>>>>>>>>>> above
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>> sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
>>>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>> Can't
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
>>>>>>>>>>> processors?
>>>>>>>>>>>>>> Then
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> able
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
>>>>>>>>>> method
>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
>>>>>>>>>>> Matthias
>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
>>>>>>>>> call
>>>>>>>>>> to
>>>>>>>>>>>>>>> aggregate
>>>>>>>>>>>>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
>>>>>>>>> Sax
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> 


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

Posted by Jeyhun Karimov <je...@gmail.com>.
Hi Matthias,

Thanks for your comments. I agree that this is not the best way to do. A
bit of history behind this design.

Prior doing this, I tried to provide ProcessorContext itself as an argument
in Rich interfaces. However, we dont want to give users that flexibility
and “power”. Moreover, ProcessorContext contains processor level
information and not Record level info. The only thing we need ij
ProcessorContext is commit() method.

So, as far as I understood, we need recor context (offset, timestamp and
etc) but also we need a commit() method ( we dont want to provide
ProcessorContext as a parameter so users can use ProcessorContext.commit()
).

As a result, I thought to “propagate” commit() call from RecordContext to
ProcessorContext() .


If there is a misunderstanding in motvation/discussion of KIP/included
jiras please let me know.


Cheers,
Jeyhun


On Fri 27. Oct 2017 at 12:39, Matthias J. Sax <ma...@confluent.io> wrote:

> I am personally still not convinced, that we should add `commit()` at all.
>
> @Guozhang: you created the original Jira. Can you elaborate a little
> bit? Isn't requesting commits a low level API that should not be exposed
> in the DSL? Just want to understand the motivation better. Why would
> anybody that uses the DSL ever want to request a commit? To me,
> requesting commits is useful if you manipulated state explicitly, ie,
> via Processor API.
>
> Also, for the solution: it seem rather unnatural to me, that we add
> `commit()` to `RecordContext` -- from my understanding, `RecordContext`
> is an helper object that provide access to record meta data. Requesting
> a commit is something quite different. Additionally, a commit does not
> commit a specific record but a `RecrodContext` is for a specific record.
>
> To me, this does not seem to be a sound API design if we follow this path.
>
>
> -Matthias
>
>
>
> On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> > Hi,
> >
> > Thanks for your suggestions.
> >
> > I have some comments, to make sure that there is no misunderstanding.
> >
> >
> > 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
> >> user to consolidate this call as
> >> "processorContext.recordContext().commit()". And internal implementation
> >> of
> >> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
> >> this call.
> >
> >
> > - I think we should not deprecate `ProcessorContext.commit()`. The main
> > intuition that we introduce `commit()` in `RecordContext` is that,
> > `RecordContext` is the one which is provided in Rich interfaces. So if
> user
> > wants to commit, then there should be some method inside `RecordContext`
> to
> > do so. Internally, `RecordContext.commit()` calls
> > `ProcessorContext.commit()`  (see the last code snippet in KIP-159):
> >
> > @Override
> >     public void process(final K1 key, final V1 value) {
> >
> >         recordContext = new RecordContext() {               //
> > recordContext initialization is added in this KIP
> >             @Override
> >             public void commit() {
> >                 context().commit();
> >             }
> >
> >             @Override
> >             public long offset() {
> >                 return context().recordContext().offset();
> >             }
> >
> >             @Override
> >             public long timestamp() {
> >                 return context().recordContext().timestamp();
> >             }
> >
> >             @Override
> >             public String topic() {
> >                 return context().recordContext().topic();
> >             }
> >
> >             @Override
> >             public int partition() {
> >                 return context().recordContext().partition();
> >             }
> >       };
> >
> >
> > So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.
> >
> >
> > 2. Add the `task` reference to the impl class, `ProcessorRecordContext`,
> so
> >> that it can implement the commit call itself.
> >
> >
> > - Actually, I don't think that we need `commit()` in
> > `ProcessorRecordContext`. 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 `RecordContext()` just only
> to
> > 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 KIP, we made `RecordContext` to be the parent class of
> > `ProcessorRecordContext`, just because of they share quite amount of
> > methods and it is logical to enable inheritance between those two.
> >
> > 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 snippet
> >> below 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 has gone though many changes
> > based on user/developer needs, both in
> documentation-/implementation-wise.
> >
> >
> > Cheers,
> > Jeyhun
> >
> >
> >
> > On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> >> Thanks for the information Jeyhun. I had also forgot about KAFKA-3907
> with
> >> this 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:
> >>
> >> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
> >> user to consolidate this call as
> >> "processorContext.recordContext().commit()". And internal implementation
> >> of
> >> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
> >> this call.
> >>
> >> 2. Add the `task` reference to the impl class,
> `ProcessorRecordContext`, so
> >> that it can implement the commit call itself.
> >>
> >> 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 snippet
> >> below would need to be updated as well.
> >>
> >>
> >> Guozhang
> >>
> >>
> >>
> >> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <matthias@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.
> >>>
> >>>
> >>> -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
> >> 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/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. Sax <
> matthias@confluent.io
> >>>
> >>>> wrote:
> >>>>
> >>>>> 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 DSL users should not worry about -- and
> >> if
> >>>>> one really needs this, an advanced user can still insert a dummy
> >>>>> `transform` to request a commit from there.
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>>>>> 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 argument to
> >>> Rich
> >>>>>> methods:
> >>>>>>
> >>>>>> instead of
> >>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>     VR apply(final V value,
> >>>>>>              final K key,
> >>>>>>              final RecordContext recordContext);
> >>>>>> }
> >>>>>>
> >>>>>> we can adopt
> >>>>>>
> >>>>>> public interface RichValueMapper<V, VR, K> {
> >>>>>>     VR apply(final V value,
> >>>>>>              final K key,
> >>>>>>              final RecordContext recordContext,
> >>>>>>              final ProcessorContext processorContext);
> >>>>>> }
> >>>>>>
> >>>>>>
> >>>>>> However, in this case, a user can get confused as ProcessorContext
> >> and
> >>>>>> RecordContext share some methods with the same name.
> >>>>>>
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Jeyhun
> >>>>>>
> >>>>>>
> >>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>>
> >>>>>>> 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 whole task up to this processed record, not only that single
> >>>>> record
> >>>>>>> itself.
> >>>>>>>
> >>>>>>>
> >>>>>>> Guozhang
> >>>>>>>
> >>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> >> je.karimov@gmail.com
> >>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>>>>>>> RecordContext
> >>>>>>>>> recordContext);
> >>>>>>>>> }
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> - Fixed.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 2. Some of the listed functions are not necessary since their
> >> pairing
> >>>>>>> APIs
> >>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> >> super
> >>> K,
> >>>>>>> ?
> >>>>>>>>> super V, KR> selector,
> >>>>>>>>>                                    final Serde<KR> keySerde,
> >>>>>>>>>                                    final Serde<V> valSerde);
> >>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>                                  final RichValueJoiner<? super K,
> >> ?
> >>>>>>> super
> >>>>>>>>> V,
> >>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Fixed
> >>>>>>>>
> >>>>>>>> 3. For a few functions where we are adding three APIs for a combo
> >> of
> >>>>> both
> >>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>>>>> subtractor,
> >>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
> >> for
> >>>>>>> both;
> >>>>>>>>> so that we can have less overloads and let users who only want to
> >>>>>>> access
> >>>>>>>>> one of them to just use dummy parameter declarations. For
> example:
> >>>>>>>>>
> >>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>> globalKTable,
> >>>>>>>>>                                  final RichKeyValueMapper<? super
> >>> K, ?
> >>>>>>>>> super
> >>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>                                  final RichValueJoiner<? super K,
> >> ?
> >>>>>>> super
> >>>>>>>>> V,
> >>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Agreed. Fixed.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> - It was a typo. Fixed.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
> >> to
> >>>>>>>>> o.a.k.processor.
> >>>>>>>>>
> >>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>> ProcessorContext
> >>>>>>> to
> >>>>>>>>> RecordContext?
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>> -
> >>>>>>>> Because it makes sense logically and  to reduce code maintenance
> >>> (both
> >>>>>>>> interfaces have offset() timestamp() topic() partition()
> >> methods),  I
> >>>>>>>> inherit ProcessorContext from RecordContext.
> >>>>>>>> Since we need commit() method both in ProcessorContext and in
> >>>>>>> RecordContext
> >>>>>>>> I move commit() method to parent class (RecordContext).
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Jeyhun
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> 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<V1, V2, VR, K> {
> >>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>>>>>>> RecordContext
> >>>>>>>>> recordContext);
> >>>>>>>>> }
> >>>>>>>>>
> >>>>>>>>> My motivation is that for lambda expression in J8, users that
> >> would
> >>>>> not
> >>>>>>>>> care about the key but only the context, or vice versa, is likely
> >> to
> >>>>>>>> write
> >>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting the
> >> dummy
> >>>>> at
> >>>>>>>> the
> >>>>>>>>> beginning of the parameter list. Generally speaking we'd like to
> >>> make
> >>>>>>> all
> >>>>>>>>> the "necessary" parameters prior to optional ones.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 2. Some of the listed functions are not necessary since their
> >>> pairing
> >>>>>>>> APIs
> >>>>>>>>> are being deprecated in 1.0 already:
> >>>>>>>>>
> >>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> >> super
> >>> K,
> >>>>>>> ?
> >>>>>>>>> super V, KR> selector,
> >>>>>>>>>                                    final Serde<KR> keySerde,
> >>>>>>>>>                                    final Serde<V> valSerde);
> >>>>>>>>>
> >>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>>>>                                  final RichValueJoiner<? super K,
> >> ?
> >>>>>>> super
> >>>>>>>>> V,
> >>>>>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>>>>                                  final Serde<K> keySerde,
> >>>>>>>>>                                  final Serde<V> valSerde);
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 3. For a few functions where we are adding three APIs for a combo
> >> of
> >>>>>>> both
> >>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>>>>> subtractor,
> >>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
> >> for
> >>>>>>> both;
> >>>>>>>>> so that we can have less overloads and let users who only want to
> >>>>>>> access
> >>>>>>>>> one of them to just use dummy parameter declarations. For
> example:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>>>>> globalKTable,
> >>>>>>>>>                                  final RichKeyValueMapper<? super
> >>> K, ?
> >>>>>>>>> super
> >>>>>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>>>>                                  final RichValueJoiner<? super K,
> >> ?
> >>>>>>> super
> >>>>>>>>> V,
> >>>>>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>>>>
> >>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> >>> initializer,
> >>>>>>>>>                              final RichAggregator<? super K, ?
> >> super
> >>>>> V,
> >>>>>>>> VR>
> >>>>>>>>> aggregator,
> >>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>> byte[]>> materialized);
> >>>>>>>>>
> >>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are
> deprecated
> >>> so
> >>>>>>> we
> >>>>>>>> do
> >>>>>>>>> not need to add its rich functions any more.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>>>>>>> Initializer also "rich" functions? I.e.
> >>>>>>>>>
> >>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
> >> K>
> >>>>>>>>> initializer,
> >>>>>>>>>                                        final RichAggregator<?
> >> super
> >>> K,
> >>>>>>> ?
> >>>>>>>>> super V, VR> aggregator);
> >>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
> >> K>
> >>>>>>>>> initializer,
> >>>>>>>>>                                        final RichAggregator<?
> >> super
> >>> K,
> >>>>>>> ?
> >>>>>>>>> super V, VR> aggregator,
> >>>>>>>>>                                        final Materialized<K, VR,
> >>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
> >> to
> >>>>>>>>> o.a.k.processor.
> >>>>>>>>>
> >>>>>>>>> 6. I'm not clear why we want to move `commit()` from
> >>> ProcessorContext
> >>>>>>> to
> >>>>>>>>> RecordContext? Conceptually I think it would better staying in
> the
> >>>>>>>>> ProcessorContext. Do you find this not doable in the internal
> >>>>>>>>> implementations?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
> >>> wrote:
> >>>>>>>>>
> >>>>>>>>>>    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 <
> >>>>>>> je.karimov@gmail.com
> >>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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 to KIP.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>>>>                              final Aggregator<? super K, ?
> >> super
> >>>>>>> V,
> >>>>>>>>> VR>
> >>>>>>>>>>>> adder,
> >>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in
> the
> >>>>>>>>> related
> >>>>>>>>>>> method. So, I had to overload all possible their Rich
> >>> counterparts:
> >>>>>>>>>>>
> >>>>>>>>>>> // adder with non-rich, subtrctor is rich
> >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>>>                              final Aggregator<? super K, ?
> super
> >>> V,
> >>>>>>>> VR>
> >>>>>>>>>>> adder,
> >>>>>>>>>>>                              final RichAggregator<? super K, ?
> >>>>>>> super
> >>>>>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> subtractor,
> >>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>
> >>>>>>>>>>> // adder withrich, subtrctor is non-rich
> >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>>>                              final RichAggregator<? super K, ?
> >>>>>>> super
> >>>>>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> adder,
> >>>>>>>>>>>                              final Aggregator<? super K, ?
> super
> >>> V,
> >>>>>>>> VR>
> >>>>>>>>>>> subtractor,
> >>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>
> >>>>>>>>>>> // both adder and subtractor are rich
> >>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>>>                              final RichAggregator<? super K, ?
> >>>>>>> super
> >>>>>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> adder,
> >>>>>>>>>>>                              final RichAggregator<? super K, ?
> >>>>>>> super
> >>>>>>>> V,
> >>>>>>>>>> VR>
> >>>>>>>>>>> subtractor,
> >>>>>>>>>>>                              final Materialized<K, VR,
> >>>>>>>>>> KeyValueStore<Bytes,
> >>>>>>>>>>> byte[]>> materialized);
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>>>>>>> supported
> >>>>>>>>>>> in
> >>>>>>>>>>>> this context");
> >>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
> >> ?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> - I added some comments both inside and outside the code
> >> snippets
> >>>>>>> in
> >>>>>>>>> KIP.
> >>>>>>>>>>> Specifically, for the code snippet above, we add *commit()*
> >> method
> >>>>>>> to
> >>>>>>>>>>> *RecordContext* interface.
> >>>>>>>>>>> However, we want  *commit()* method to be used only for
> >>>>>>>> *RecordContext*
> >>>>>>>>>>> instances (at least for now), so we add
> >>>>>>> UnsupportedOperationException
> >>>>>>>>> in
> >>>>>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
> >>>>>>>>>>> In general, 1) we make RecordContext publicly available within
> >>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
> >> required
> >>>>>>>>>>> Processors and 3) pass it as an argument to the related Rich
> >>>>>>>> interfaces
> >>>>>>>>>>> inside Processors.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Jeyhun
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> >>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> 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.
> >>>>>>>>>>>>
> >>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR>
> initializer,
> >>>>>>>>>>>>                              final Aggregator<? super K, ?
> >> super
> >>>>>>> V,
> >>>>>>>>> VR>
> >>>>>>>>>>>> adder,
> >>>>>>>>>>>>
> >>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>>>>
> >>>>>>>>>>>>   public RecordContext recordContext() {
> >>>>>>>>>>>>     return this.recordContext();
> >>>>>>>>>>>>
> >>>>>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>>>>
> >>>>>>>>>>>>    void commit () {
> >>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>>>>>>> supported
> >>>>>>>>>>> in
> >>>>>>>>>>>> this context");
> >>>>>>>>>>>>
> >>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
> >> ?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Cheers
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> >>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 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 <
> >>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> 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 <
> >>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>>> 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 Jeyhun Karimov <
> >>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
> >>>>>>>> not
> >>>>>>>>>>> voted
> >>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> >>>>>>>> "atomic"
> >>>>>>>>>>>> merge).
> >>>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> >>>>>>> some
> >>>>>>>>>> minor
> >>>>>>>>>>>>>>> updates
> >>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
> >>>>>>> if
> >>>>>>>> I
> >>>>>>>>> am
> >>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>> or I
> >>>>>>>>>>>>>>>> misunderstood.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> >>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> >>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>> 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
> >>>>>>>> would
> >>>>>>>>>>> still
> >>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>> update your proposal for people to comment and vote
> >>>>>>> on.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> >>>>>>>>>> yuzhihong@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> One interface is cleaner.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> >>>>>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> >>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>> interfaces
> >>>>>>>>>>>>>>>>>>> into 1
> >>>>>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 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
> >>>>>>> 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
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> would not be a problem.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> >>>>>>>>>>>> implements
> >>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> >>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> >>>>>>>>> RichXXX
> >>>>>>>>>>> or
> >>>>>>>>>>>>>>>>> XXXWithKey
> >>>>>>>>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> >>>>>>> to
> >>>>>>>>>> revert
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>> to this again.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >>>>>>>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> >>>>>>> was
> >>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> >>>>>>> RichXXXX
> >>>>>>>>> and
> >>>>>>>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>>>>> 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?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>>>>>>> confluence/pages/viewpage.
> >>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> >>>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> >>>>>>>> issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> >>>>>>>> Guy
> >>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>> overrides.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> >>>>>>>> RecordContext,
> >>>>>>>>>> you
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>>> right.
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> >>>>>>> the
> >>>>>>>>>> full
> >>>>>>>>>>>>>>> picture
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>> parts
> >>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> >>>>>>> as
> >>>>>>>> I
> >>>>>>>>>>>> thought.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> >>>>>>> Damian
> >>>>>>>>> Guy
> >>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
> >>>>>>>> to
> >>>>>>>>>>> cast a
> >>>>>>>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
> >>>>>>>>>>>>>>>>> InternalProcessorContext
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> init()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
> >>>>>>>> and i
> >>>>>>>>>>> think
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>> remain
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
> >>>>>>>>>>> recordContext()
> >>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
> >>>>>>>>>>> ProcessorContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
> >>>>>>>> showing:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
> >>>>>>>>>>>>> processorContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
> >>>>>>>> VR>
> >>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
> >>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
> >>>>>>>> as
> >>>>>>>>>>> above
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>> sense.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> >>>>>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>> Can't
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> pass
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
> >>>>>>>>> processors?
> >>>>>>>>>>>> Then
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>> able
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
> >>>>>>>> method
> >>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
> >>>>>>>>> Matthias
> >>>>>>>>>>> J.
> >>>>>>>>>>>>> Sax
> >>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
> >>>>>>> call
> >>>>>>>> to
> >>>>>>>>>>>>> aggregate
> >>>>>>>>>>>>>>>>>> anyways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
> >>>>>>> Sax
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I am personally still not convinced, that we should add `commit()` at all.

@Guozhang: you created the original Jira. Can you elaborate a little
bit? Isn't requesting commits a low level API that should not be exposed
in the DSL? Just want to understand the motivation better. Why would
anybody that uses the DSL ever want to request a commit? To me,
requesting commits is useful if you manipulated state explicitly, ie,
via Processor API.

Also, for the solution: it seem rather unnatural to me, that we add
`commit()` to `RecordContext` -- from my understanding, `RecordContext`
is an helper object that provide access to record meta data. Requesting
a commit is something quite different. Additionally, a commit does not
commit a specific record but a `RecrodContext` is for a specific record.

To me, this does not seem to be a sound API design if we follow this path.


-Matthias



On 10/26/17 10:41 PM, Jeyhun Karimov wrote:
> Hi,
> 
> Thanks for your suggestions.
> 
> I have some comments, to make sure that there is no misunderstanding.
> 
> 
> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
>> user to consolidate this call as
>> "processorContext.recordContext().commit()". And internal implementation
>> of
>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
>> this call.
> 
> 
> - I think we should not deprecate `ProcessorContext.commit()`. The main
> intuition that we introduce `commit()` in `RecordContext` is that,
> `RecordContext` is the one which is provided in Rich interfaces. So if user
> wants to commit, then there should be some method inside `RecordContext` to
> do so. Internally, `RecordContext.commit()` calls
> `ProcessorContext.commit()`  (see the last code snippet in KIP-159):
> 
> @Override
>     public void process(final K1 key, final V1 value) {
> 
>         recordContext = new RecordContext() {               //
> recordContext initialization is added in this KIP
>             @Override
>             public void commit() {
>                 context().commit();
>             }
> 
>             @Override
>             public long offset() {
>                 return context().recordContext().offset();
>             }
> 
>             @Override
>             public long timestamp() {
>                 return context().recordContext().timestamp();
>             }
> 
>             @Override
>             public String topic() {
>                 return context().recordContext().topic();
>             }
> 
>             @Override
>             public int partition() {
>                 return context().recordContext().partition();
>             }
>       };
> 
> 
> So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.
> 
> 
> 2. Add the `task` reference to the impl class, `ProcessorRecordContext`, so
>> that it can implement the commit call itself.
> 
> 
> - Actually, I don't think that we need `commit()` in
> `ProcessorRecordContext`. 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 `RecordContext()` just only to
> 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 KIP, we made `RecordContext` to be the parent class of
> `ProcessorRecordContext`, just because of they share quite amount of
> methods and it is logical to enable inheritance between those two.
> 
> 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 snippet
>> below 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 has gone though many changes
> based on user/developer needs, both in documentation-/implementation-wise.
> 
> 
> Cheers,
> Jeyhun
> 
> 
> 
> On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com> wrote:
> 
>> Thanks for the information Jeyhun. I had also forgot about KAFKA-3907 with
>> this 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:
>>
>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
>> user to consolidate this call as
>> "processorContext.recordContext().commit()". And internal implementation
>> of
>> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
>> this call.
>>
>> 2. Add the `task` reference to the impl class, `ProcessorRecordContext`, so
>> that it can implement the commit call itself.
>>
>> 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 snippet
>> below would need to be updated as well.
>>
>>
>> Guozhang
>>
>>
>>
>> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <ma...@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.
>>>
>>>
>>> -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
>> 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/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. Sax <matthias@confluent.io
>>>
>>>> wrote:
>>>>
>>>>> 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 DSL users should not worry about -- and
>> if
>>>>> one really needs this, an advanced user can still insert a dummy
>>>>> `transform` to request a commit from there.
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>>>>> 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 argument to
>>> Rich
>>>>>> methods:
>>>>>>
>>>>>> instead of
>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>     VR apply(final V value,
>>>>>>              final K key,
>>>>>>              final RecordContext recordContext);
>>>>>> }
>>>>>>
>>>>>> we can adopt
>>>>>>
>>>>>> public interface RichValueMapper<V, VR, K> {
>>>>>>     VR apply(final V value,
>>>>>>              final K key,
>>>>>>              final RecordContext recordContext,
>>>>>>              final ProcessorContext processorContext);
>>>>>> }
>>>>>>
>>>>>>
>>>>>> However, in this case, a user can get confused as ProcessorContext
>> and
>>>>>> RecordContext share some methods with the same name.
>>>>>>
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>>
>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>
>>>>>>
>>>>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>>
>>>>>>> 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 whole task up to this processed record, not only that single
>>>>> record
>>>>>>> itself.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
>> je.karimov@gmail.com
>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>>>>> RecordContext
>>>>>>>>> recordContext);
>>>>>>>>> }
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> - Fixed.
>>>>>>>>
>>>>>>>>
>>>>>>>> 2. Some of the listed functions are not necessary since their
>> pairing
>>>>>>> APIs
>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
>> super
>>> K,
>>>>>>> ?
>>>>>>>>> super V, KR> selector,
>>>>>>>>>                                    final Serde<KR> keySerde,
>>>>>>>>>                                    final Serde<V> valSerde);
>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>                                  final RichValueJoiner<? super K,
>> ?
>>>>>>> super
>>>>>>>>> V,
>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>
>>>>>>>>
>>>>>>>> -Fixed
>>>>>>>>
>>>>>>>> 3. For a few functions where we are adding three APIs for a combo
>> of
>>>>> both
>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>>>>> subtractor,
>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
>> for
>>>>>>> both;
>>>>>>>>> so that we can have less overloads and let users who only want to
>>>>>>> access
>>>>>>>>> one of them to just use dummy parameter declarations. For example:
>>>>>>>>>
>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>> globalKTable,
>>>>>>>>>                                  final RichKeyValueMapper<? super
>>> K, ?
>>>>>>>>> super
>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>                                  final RichValueJoiner<? super K,
>> ?
>>>>>>> super
>>>>>>>>> V,
>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Agreed. Fixed.
>>>>>>>>
>>>>>>>>
>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>
>>>>>>>>
>>>>>>>> - It was a typo. Fixed.
>>>>>>>>
>>>>>>>>
>>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
>> to
>>>>>>>>> o.a.k.processor.
>>>>>>>>>
>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>> ProcessorContext
>>>>>>> to
>>>>>>>>> RecordContext?
>>>>>>>>>
>>>>>>>>
>>>>>>>> -
>>>>>>>> Because it makes sense logically and  to reduce code maintenance
>>> (both
>>>>>>>> interfaces have offset() timestamp() topic() partition()
>> methods),  I
>>>>>>>> inherit ProcessorContext from RecordContext.
>>>>>>>> Since we need commit() method both in ProcessorContext and in
>>>>>>> RecordContext
>>>>>>>> I move commit() method to parent class (RecordContext).
>>>>>>>>
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
>> wangguoz@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> 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<V1, V2, VR, K> {
>>>>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>>>>> RecordContext
>>>>>>>>> recordContext);
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> My motivation is that for lambda expression in J8, users that
>> would
>>>>> not
>>>>>>>>> care about the key but only the context, or vice versa, is likely
>> to
>>>>>>>> write
>>>>>>>>> it as (value1, value2, dummy, context) -> ... than putting the
>> dummy
>>>>> at
>>>>>>>> the
>>>>>>>>> beginning of the parameter list. Generally speaking we'd like to
>>> make
>>>>>>> all
>>>>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2. Some of the listed functions are not necessary since their
>>> pairing
>>>>>>>> APIs
>>>>>>>>> are being deprecated in 1.0 already:
>>>>>>>>>
>>>>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
>> super
>>> K,
>>>>>>> ?
>>>>>>>>> super V, KR> selector,
>>>>>>>>>                                    final Serde<KR> keySerde,
>>>>>>>>>                                    final Serde<V> valSerde);
>>>>>>>>>
>>>>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>>>>                                  final RichValueJoiner<? super K,
>> ?
>>>>>>> super
>>>>>>>>> V,
>>>>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>>>>                                  final Serde<K> keySerde,
>>>>>>>>>                                  final Serde<V> valSerde);
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 3. For a few functions where we are adding three APIs for a combo
>> of
>>>>>>> both
>>>>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>>>>> subtractor,
>>>>>>>>> I'm wondering if we can just keep one that use "rich" functions
>> for
>>>>>>> both;
>>>>>>>>> so that we can have less overloads and let users who only want to
>>>>>>> access
>>>>>>>>> one of them to just use dummy parameter declarations. For example:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>>>>> globalKTable,
>>>>>>>>>                                  final RichKeyValueMapper<? super
>>> K, ?
>>>>>>>>> super
>>>>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>>>>                                  final RichValueJoiner<? super K,
>> ?
>>>>>>> super
>>>>>>>>> V,
>>>>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>>>>
>>>>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
>>> initializer,
>>>>>>>>>                              final RichAggregator<? super K, ?
>> super
>>>>> V,
>>>>>>>> VR>
>>>>>>>>> aggregator,
>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>> byte[]>> materialized);
>>>>>>>>>
>>>>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated
>>> so
>>>>>>> we
>>>>>>>> do
>>>>>>>>> not need to add its rich functions any more.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>>>>> Initializer also "rich" functions? I.e.
>>>>>>>>>
>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
>> K>
>>>>>>>>> initializer,
>>>>>>>>>                                        final RichAggregator<?
>> super
>>> K,
>>>>>>> ?
>>>>>>>>> super V, VR> aggregator);
>>>>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
>> K>
>>>>>>>>> initializer,
>>>>>>>>>                                        final RichAggregator<?
>> super
>>> K,
>>>>>>> ?
>>>>>>>>> super V, VR> aggregator,
>>>>>>>>>                                        final Materialized<K, VR,
>>>>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
>> to
>>>>>>>>> o.a.k.processor.
>>>>>>>>>
>>>>>>>>> 6. I'm not clear why we want to move `commit()` from
>>> ProcessorContext
>>>>>>> to
>>>>>>>>> RecordContext? Conceptually I think it would better staying in the
>>>>>>>>> ProcessorContext. Do you find this not doable in the internal
>>>>>>>>> implementations?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
>>> wrote:
>>>>>>>>>
>>>>>>>>>>    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 <
>>>>>>> je.karimov@gmail.com
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 to KIP.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>                              final Aggregator<? super K, ?
>> super
>>>>>>> V,
>>>>>>>>> VR>
>>>>>>>>>>>> adder,
>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
>>>>>>>>> related
>>>>>>>>>>> method. So, I had to overload all possible their Rich
>>> counterparts:
>>>>>>>>>>>
>>>>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>                              final Aggregator<? super K, ? super
>>> V,
>>>>>>>> VR>
>>>>>>>>>>> adder,
>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>> super
>>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>> subtractor,
>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>
>>>>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>> super
>>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>> adder,
>>>>>>>>>>>                              final Aggregator<? super K, ? super
>>> V,
>>>>>>>> VR>
>>>>>>>>>>> subtractor,
>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>
>>>>>>>>>>> // both adder and subtractor are rich
>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>> super
>>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>> adder,
>>>>>>>>>>>                              final RichAggregator<? super K, ?
>>>>>>> super
>>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>> subtractor,
>>>>>>>>>>>                              final Materialized<K, VR,
>>>>>>>>>> KeyValueStore<Bytes,
>>>>>>>>>>> byte[]>> materialized);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>>>>> supported
>>>>>>>>>>> in
>>>>>>>>>>>> this context");
>>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
>> ?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> - I added some comments both inside and outside the code
>> snippets
>>>>>>> in
>>>>>>>>> KIP.
>>>>>>>>>>> Specifically, for the code snippet above, we add *commit()*
>> method
>>>>>>> to
>>>>>>>>>>> *RecordContext* interface.
>>>>>>>>>>> However, we want  *commit()* method to be used only for
>>>>>>>> *RecordContext*
>>>>>>>>>>> instances (at least for now), so we add
>>>>>>> UnsupportedOperationException
>>>>>>>>> in
>>>>>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
>>>>>>>>>>> In general, 1) we make RecordContext publicly available within
>>>>>>>>>>> ProcessorContext,  2) initialize its instance within all
>> required
>>>>>>>>>>> Processors and 3) pass it as an argument to the related Rich
>>>>>>>> interfaces
>>>>>>>>>>> inside Processors.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> 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.
>>>>>>>>>>>>
>>>>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>>>>                              final Aggregator<? super K, ?
>> super
>>>>>>> V,
>>>>>>>>> VR>
>>>>>>>>>>>> adder,
>>>>>>>>>>>>
>>>>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>>>>
>>>>>>>>>>>>   public RecordContext recordContext() {
>>>>>>>>>>>>     return this.recordContext();
>>>>>>>>>>>>
>>>>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>>>>
>>>>>>>>>>>>    void commit () {
>>>>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>>>>> supported
>>>>>>>>>>> in
>>>>>>>>>>>> this context");
>>>>>>>>>>>>
>>>>>>>>>>>> Is the exception going to be replaced with real code in the PR
>> ?
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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 <
>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 <
>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>
>>>>>>>>>>>> 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 Jeyhun Karimov <
>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
>>>>>>>> not
>>>>>>>>>>> voted
>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
>>>>>>>> "atomic"
>>>>>>>>>>>> merge).
>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
>>>>>>> some
>>>>>>>>>> minor
>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
>>>>>>> if
>>>>>>>> I
>>>>>>>>> am
>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>> or I
>>>>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>> 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
>>>>>>>> would
>>>>>>>>>>> still
>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>> update your proposal for people to comment and vote
>>>>>>> on.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>>>>> yuzhihong@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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
>>>>>>> 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
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>>>>> RichXXX
>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>>>>> to
>>>>>>>>>> revert
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>>>>> was
>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>>>>> RichXXXX
>>>>>>>>> and
>>>>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
>>>>>>>> issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
>>>>>>>> Guy
>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>> overrides.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
>>>>>>>> RecordContext,
>>>>>>>>>> you
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> right.
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
>>>>>>> the
>>>>>>>>>> full
>>>>>>>>>>>>>>> picture
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> parts
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
>>>>>>> as
>>>>>>>> I
>>>>>>>>>>>> thought.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
>>>>>>> Damian
>>>>>>>>> Guy
>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
>>>>>>>> to
>>>>>>>>>>> cast a
>>>>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
>>>>>>>>>>>>>>>>> InternalProcessorContext
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> init()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
>>>>>>>> and i
>>>>>>>>>>> think
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>> remain
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
>>>>>>>>>>> recordContext()
>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
>>>>>>>>>>> ProcessorContext.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
>>>>>>>> showing:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
>>>>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
>>>>>>>> VR>
>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
>>>>>>>> as
>>>>>>>>>>> above
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>> sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
>>>>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>> Can't
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
>>>>>>>>> processors?
>>>>>>>>>>>> Then
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> able
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
>>>>>>>> method
>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
>>>>>>>>> Matthias
>>>>>>>>>>> J.
>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
>>>>>>> call
>>>>>>>> to
>>>>>>>>>>>>> aggregate
>>>>>>>>>>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
>>>>>>> Sax
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>> implement
>>>>>>>>>>>>>>> both.
>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>> allows
>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cast
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
>>>>>>>> limit
>>>>>>>>>> the
>>>>>>>>>>>>>>> visible
>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
>>>>>>> Karimov
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
>>>>>>>> as
>>>>>>>>>> well
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
>>>>>>> PM,
>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>> Karimov
>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>> KAFKA-3907
>>>>>>>>>>> [1]
>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> integrated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
>>>>>>>>> inside
>>>>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
>>>>>>> PM
>>>>>>>>>>> Matthias
>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to push this
>>>>>>>>>> discussion
>>>>>>>>>>>>>>> further.
>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> got
>>>>>>>>>>>>>>>>
> 


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

Posted by Jeyhun Karimov <je...@gmail.com>.
Hi,

Thanks for your suggestions.

I have some comments, to make sure that there is no misunderstanding.


1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
> user to consolidate this call as
> "processorContext.recordContext().commit()". And internal implementation
> of
> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
> this call.


- I think we should not deprecate `ProcessorContext.commit()`. The main
intuition that we introduce `commit()` in `RecordContext` is that,
`RecordContext` is the one which is provided in Rich interfaces. So if user
wants to commit, then there should be some method inside `RecordContext` to
do so. Internally, `RecordContext.commit()` calls
`ProcessorContext.commit()`  (see the last code snippet in KIP-159):

@Override
    public void process(final K1 key, final V1 value) {

        recordContext = new RecordContext() {               //
recordContext initialization is added in this KIP
            @Override
            public void commit() {
                context().commit();
            }

            @Override
            public long offset() {
                return context().recordContext().offset();
            }

            @Override
            public long timestamp() {
                return context().recordContext().timestamp();
            }

            @Override
            public String topic() {
                return context().recordContext().topic();
            }

            @Override
            public int partition() {
                return context().recordContext().partition();
            }
      };


So, we cannot deprecate `ProcessorContext.commit()` in this case IMO.


2. Add the `task` reference to the impl class, `ProcessorRecordContext`, so
> that it can implement the commit call itself.


- Actually, I don't think that we need `commit()` in
`ProcessorRecordContext`. 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 `RecordContext()` just only to
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 KIP, we made `RecordContext` to be the parent class of
`ProcessorRecordContext`, just because of they share quite amount of
methods and it is logical to enable inheritance between those two.

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 snippet
> below 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 has gone though many changes
based on user/developer needs, both in documentation-/implementation-wise.


Cheers,
Jeyhun



On Tue, Oct 24, 2017 at 1:41 AM Guozhang Wang <wa...@gmail.com> wrote:

> Thanks for the information Jeyhun. I had also forgot about KAFKA-3907 with
> this 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:
>
> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
> user to consolidate this call as
> "processorContext.recordContext().commit()". And internal implementation
> of
> `ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
> this call.
>
> 2. Add the `task` reference to the impl class, `ProcessorRecordContext`, so
> that it can implement the commit call itself.
>
> 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 snippet
> below would need to be updated as well.
>
>
> Guozhang
>
>
>
> On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <ma...@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.
> >
> >
> > -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
> 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/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. Sax <matthias@confluent.io
> >
> > > wrote:
> > >
> > >> 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 DSL users should not worry about -- and
> if
> > >> one really needs this, an advanced user can still insert a dummy
> > >> `transform` to request a commit from there.
> > >>
> > >> -Matthias
> > >>
> > >>
> > >> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> > >>> 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 argument to
> > Rich
> > >>> methods:
> > >>>
> > >>> instead of
> > >>> public interface RichValueMapper<V, VR, K> {
> > >>>     VR apply(final V value,
> > >>>              final K key,
> > >>>              final RecordContext recordContext);
> > >>> }
> > >>>
> > >>> we can adopt
> > >>>
> > >>> public interface RichValueMapper<V, VR, K> {
> > >>>     VR apply(final V value,
> > >>>              final K key,
> > >>>              final RecordContext recordContext,
> > >>>              final ProcessorContext processorContext);
> > >>> }
> > >>>
> > >>>
> > >>> However, in this case, a user can get confused as ProcessorContext
> and
> > >>> RecordContext share some methods with the same name.
> > >>>
> > >>>
> > >>> Cheers,
> > >>> Jeyhun
> > >>>
> > >>>
> > >>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > >>>
> > >>>
> > >>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >>>
> > >>>> 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 whole task up to this processed record, not only that single
> > >> record
> > >>>> itself.
> > >>>>
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > >>>> wrote:
> > >>>>
> > >>>>> 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<V1, V2, VR, K> {
> > >>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> > >>>>>> RecordContext
> > >>>>>> recordContext);
> > >>>>>> }
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> - Fixed.
> > >>>>>
> > >>>>>
> > >>>>> 2. Some of the listed functions are not necessary since their
> pairing
> > >>>> APIs
> > >>>>>> are being deprecated in 1.0 already:
> > >>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> super
> > K,
> > >>>> ?
> > >>>>>> super V, KR> selector,
> > >>>>>>                                    final Serde<KR> keySerde,
> > >>>>>>                                    final Serde<V> valSerde);
> > >>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >>>>>>                                  final RichValueJoiner<? super K,
> ?
> > >>>> super
> > >>>>>> V,
> > >>>>>> ? super VT, ? extends VR> joiner,
> > >>>>>>                                  final Serde<K> keySerde,
> > >>>>>>                                  final Serde<V> valSerde);
> > >>>>>
> > >>>>>
> > >>>>> -Fixed
> > >>>>>
> > >>>>> 3. For a few functions where we are adding three APIs for a combo
> of
> > >> both
> > >>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> > >>>> subtractor,
> > >>>>>> I'm wondering if we can just keep one that use "rich" functions
> for
> > >>>> both;
> > >>>>>> so that we can have less overloads and let users who only want to
> > >>>> access
> > >>>>>> one of them to just use dummy parameter declarations. For example:
> > >>>>>>
> > >>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > >>>> globalKTable,
> > >>>>>>                                  final RichKeyValueMapper<? super
> > K, ?
> > >>>>>> super
> > >>>>>>  V, ? extends GK> keyValueMapper,
> > >>>>>>                                  final RichValueJoiner<? super K,
> ?
> > >>>> super
> > >>>>>> V,
> > >>>>>> ? super GV, ? extends RV> joiner);
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> -Agreed. Fixed.
> > >>>>>
> > >>>>>
> > >>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > >>>>>> Initializer also "rich" functions? I.e.
> > >>>>>
> > >>>>>
> > >>>>> - It was a typo. Fixed.
> > >>>>>
> > >>>>>
> > >>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
> to
> > >>>>>> o.a.k.processor.
> > >>>>>>
> > >>>>>> 6. I'm not clear why we want to move `commit()` from
> > ProcessorContext
> > >>>> to
> > >>>>>> RecordContext?
> > >>>>>>
> > >>>>>
> > >>>>> -
> > >>>>> Because it makes sense logically and  to reduce code maintenance
> > (both
> > >>>>> interfaces have offset() timestamp() topic() partition()
> methods),  I
> > >>>>> inherit ProcessorContext from RecordContext.
> > >>>>> Since we need commit() method both in ProcessorContext and in
> > >>>> RecordContext
> > >>>>> I move commit() method to parent class (RecordContext).
> > >>>>>
> > >>>>>
> > >>>>> Cheers,
> > >>>>> Jeyhun
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <
> wangguoz@gmail.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> 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<V1, V2, VR, K> {
> > >>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> > >>>>>> RecordContext
> > >>>>>> recordContext);
> > >>>>>> }
> > >>>>>>
> > >>>>>> My motivation is that for lambda expression in J8, users that
> would
> > >> not
> > >>>>>> care about the key but only the context, or vice versa, is likely
> to
> > >>>>> write
> > >>>>>> it as (value1, value2, dummy, context) -> ... than putting the
> dummy
> > >> at
> > >>>>> the
> > >>>>>> beginning of the parameter list. Generally speaking we'd like to
> > make
> > >>>> all
> > >>>>>> the "necessary" parameters prior to optional ones.
> > >>>>>>
> > >>>>>>
> > >>>>>> 2. Some of the listed functions are not necessary since their
> > pairing
> > >>>>> APIs
> > >>>>>> are being deprecated in 1.0 already:
> > >>>>>>
> > >>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<?
> super
> > K,
> > >>>> ?
> > >>>>>> super V, KR> selector,
> > >>>>>>                                    final Serde<KR> keySerde,
> > >>>>>>                                    final Serde<V> valSerde);
> > >>>>>>
> > >>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >>>>>>                                  final RichValueJoiner<? super K,
> ?
> > >>>> super
> > >>>>>> V,
> > >>>>>> ? super VT, ? extends VR> joiner,
> > >>>>>>                                  final Serde<K> keySerde,
> > >>>>>>                                  final Serde<V> valSerde);
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> 3. For a few functions where we are adding three APIs for a combo
> of
> > >>>> both
> > >>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> > >>>> subtractor,
> > >>>>>> I'm wondering if we can just keep one that use "rich" functions
> for
> > >>>> both;
> > >>>>>> so that we can have less overloads and let users who only want to
> > >>>> access
> > >>>>>> one of them to just use dummy parameter declarations. For example:
> > >>>>>>
> > >>>>>>
> > >>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> > >>>> globalKTable,
> > >>>>>>                                  final RichKeyValueMapper<? super
> > K, ?
> > >>>>>> super
> > >>>>>>  V, ? extends GK> keyValueMapper,
> > >>>>>>                                  final RichValueJoiner<? super K,
> ?
> > >>>> super
> > >>>>>> V,
> > >>>>>> ? super GV, ? extends RV> joiner);
> > >>>>>>
> > >>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> > initializer,
> > >>>>>>                              final RichAggregator<? super K, ?
> super
> > >> V,
> > >>>>> VR>
> > >>>>>> aggregator,
> > >>>>>>                              final Materialized<K, VR,
> > >>>>> KeyValueStore<Bytes,
> > >>>>>> byte[]>> materialized);
> > >>>>>>
> > >>>>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated
> > so
> > >>>> we
> > >>>>> do
> > >>>>>> not need to add its rich functions any more.
> > >>>>>>
> > >>>>>>
> > >>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > >>>>>> Initializer also "rich" functions? I.e.
> > >>>>>>
> > >>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
> K>
> > >>>>>> initializer,
> > >>>>>>                                        final RichAggregator<?
> super
> > K,
> > >>>> ?
> > >>>>>> super V, VR> aggregator);
> > >>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR,
> K>
> > >>>>>> initializer,
> > >>>>>>                                        final RichAggregator<?
> super
> > K,
> > >>>> ?
> > >>>>>> super V, VR> aggregator,
> > >>>>>>                                        final Materialized<K, VR,
> > >>>>>> WindowStore<Bytes, byte[]>> materialized);
> > >>>>>>
> > >>>>>>
> > >>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals
> to
> > >>>>>> o.a.k.processor.
> > >>>>>>
> > >>>>>> 6. I'm not clear why we want to move `commit()` from
> > ProcessorContext
> > >>>> to
> > >>>>>> RecordContext? Conceptually I think it would better staying in the
> > >>>>>> ProcessorContext. Do you find this not doable in the internal
> > >>>>>> implementations?
> > >>>>>>
> > >>>>>>
> > >>>>>> Guozhang
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
> > wrote:
> > >>>>>>
> > >>>>>>>    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 <
> > >>>> je.karimov@gmail.com
> > >>>>>>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> 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 to KIP.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >>>>>>>>>                              final Aggregator<? super K, ?
> super
> > >>>> V,
> > >>>>>> VR>
> > >>>>>>>>> adder,
> > >>>>>>>>> The adder doesn't need to be RichAggregator ?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
> > >>>>>> related
> > >>>>>>>> method. So, I had to overload all possible their Rich
> > counterparts:
> > >>>>>>>>
> > >>>>>>>> // adder with non-rich, subtrctor is rich
> > >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >>>>>>>>                              final Aggregator<? super K, ? super
> > V,
> > >>>>> VR>
> > >>>>>>>> adder,
> > >>>>>>>>                              final RichAggregator<? super K, ?
> > >>>> super
> > >>>>> V,
> > >>>>>>> VR>
> > >>>>>>>> subtractor,
> > >>>>>>>>                              final Materialized<K, VR,
> > >>>>>>> KeyValueStore<Bytes,
> > >>>>>>>> byte[]>> materialized);
> > >>>>>>>>
> > >>>>>>>> // adder withrich, subtrctor is non-rich
> > >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >>>>>>>>                              final RichAggregator<? super K, ?
> > >>>> super
> > >>>>> V,
> > >>>>>>> VR>
> > >>>>>>>> adder,
> > >>>>>>>>                              final Aggregator<? super K, ? super
> > V,
> > >>>>> VR>
> > >>>>>>>> subtractor,
> > >>>>>>>>                              final Materialized<K, VR,
> > >>>>>>> KeyValueStore<Bytes,
> > >>>>>>>> byte[]>> materialized);
> > >>>>>>>>
> > >>>>>>>> // both adder and subtractor are rich
> > >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >>>>>>>>                              final RichAggregator<? super K, ?
> > >>>> super
> > >>>>> V,
> > >>>>>>> VR>
> > >>>>>>>> adder,
> > >>>>>>>>                              final RichAggregator<? super K, ?
> > >>>> super
> > >>>>> V,
> > >>>>>>> VR>
> > >>>>>>>> subtractor,
> > >>>>>>>>                              final Materialized<K, VR,
> > >>>>>>> KeyValueStore<Bytes,
> > >>>>>>>> byte[]>> materialized);
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Can you explain a bit about the above implementation ?
> > >>>>>>>>>    void commit () {
> > >>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> > >>>>>>> supported
> > >>>>>>>> in
> > >>>>>>>>> this context");
> > >>>>>>>>> Is the exception going to be replaced with real code in the PR
> ?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> - I added some comments both inside and outside the code
> snippets
> > >>>> in
> > >>>>>> KIP.
> > >>>>>>>> Specifically, for the code snippet above, we add *commit()*
> method
> > >>>> to
> > >>>>>>>> *RecordContext* interface.
> > >>>>>>>> However, we want  *commit()* method to be used only for
> > >>>>> *RecordContext*
> > >>>>>>>> instances (at least for now), so we add
> > >>>> UnsupportedOperationException
> > >>>>>> in
> > >>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
> > >>>>>>>> In general, 1) we make RecordContext publicly available within
> > >>>>>>>> ProcessorContext,  2) initialize its instance within all
> required
> > >>>>>>>> Processors and 3) pass it as an argument to the related Rich
> > >>>>> interfaces
> > >>>>>>>> inside Processors.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Jeyhun
> > >>>>>>>>
> > >>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> > >>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> 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.
> > >>>>>>>>>
> > >>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >>>>>>>>>                              final Aggregator<? super K, ?
> super
> > >>>> V,
> > >>>>>> VR>
> > >>>>>>>>> adder,
> > >>>>>>>>>
> > >>>>>>>>> The adder doesn't need to be RichAggregator ?
> > >>>>>>>>>
> > >>>>>>>>>   public RecordContext recordContext() {
> > >>>>>>>>>     return this.recordContext();
> > >>>>>>>>>
> > >>>>>>>>> Can you explain a bit about the above implementation ?
> > >>>>>>>>>
> > >>>>>>>>>    void commit () {
> > >>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> > >>>>>>> supported
> > >>>>>>>> in
> > >>>>>>>>> this context");
> > >>>>>>>>>
> > >>>>>>>>> Is the exception going to be replaced with real code in the PR
> ?
> > >>>>>>>>>
> > >>>>>>>>> Cheers
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> > >>>>>> je.karimov@gmail.com>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> 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 <
> > >>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> 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 <
> > >>>>> damian.guy@gmail.com
> > >>>>>>>
> > >>>>>>>>> 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 Jeyhun Karimov <
> > >>>>>> je.karimov@gmail.com
> > >>>>>>>>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
> > >>>>> not
> > >>>>>>>> voted
> > >>>>>>>>>> yet
> > >>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> > >>>>> "atomic"
> > >>>>>>>>> merge).
> > >>>>>>>>>>>> So
> > >>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> > >>>> some
> > >>>>>>> minor
> > >>>>>>>>>>>> updates
> > >>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
> > >>>> if
> > >>>>> I
> > >>>>>> am
> > >>>>>>>>> wrong
> > >>>>>>>>>>>> or I
> > >>>>>>>>>>>>> misunderstood.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > >>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> +1
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > >>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>> 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
> > >>>>> would
> > >>>>>>>> still
> > >>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>> update your proposal for people to comment and vote
> > >>>> on.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > >>>>>>> yuzhihong@gmail.com
> > >>>>>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> +1
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> One interface is cleaner.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > >>>>>>>>> bbejeck@gmail.com
> > >>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> > >>>>>>> ValueXXXXWithKey
> > >>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>>> into 1
> > >>>>>>>>>>>>>>>>> interface.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>> Bill
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > >>>>>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> 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
> > >>>> 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
> > >>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>> would not be a problem.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> > >>>>>>>>> implements
> > >>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> > >>>>>>>>>>>>>>>>>> ......
> > >>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> > >>>>>> RichXXX
> > >>>>>>>> or
> > >>>>>>>>>>>>>> XXXWithKey
> > >>>>>>>>>>>>>>>>> inside
> > >>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> > >>>> to
> > >>>>>>> revert
> > >>>>>>>>> the
> > >>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>> design
> > >>>>>>>>>>>>>>>>>> to this again.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > >>>>>>>>>>>> damian.guy@gmail.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> > >>>> was
> > >>>>>>>>> thinking
> > >>>>>>>>>>>> more
> > >>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> > >>>> RichXXXX
> > >>>>>> and
> > >>>>>>>>>>>>>>> ValueXXXXWithKey
> > >>>>>>>>>>>>>>>>> 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?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > >>>>>>>>>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > >>>>>> confluence/pages/viewpage.
> > >>>>>>>>>>>>>>>>>> action?pageId=73637757
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> > >>>>> Karimov <
> > >>>>>>>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>>>>>>>>>> matthias@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> > >>>>> issue.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> > >>>>> Guy
> > >>>>>> <
> > >>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>> overrides.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> > >>>>> RecordContext,
> > >>>>>>> you
> > >>>>>>>>> are
> > >>>>>>>>>>>>> right.
> > >>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> > >>>> the
> > >>>>>>> full
> > >>>>>>>>>>>> picture
> > >>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>> parts
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> > >>>> as
> > >>>>> I
> > >>>>>>>>> thought.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> > >>>> Damian
> > >>>>>> Guy
> > >>>>>>> <
> > >>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>> lifetime
> > >>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
> > >>>>> to
> > >>>>>>>> cast a
> > >>>>>>>>>>>>>>>>> ProcessorContext
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
> > >>>>>>>>>>>>>> InternalProcessorContext
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> init()
> > >>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
> > >>>>> and i
> > >>>>>>>> think
> > >>>>>>>>>> it
> > >>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>> remain
> > >>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>> way.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
> > >>>>>>>> recordContext()
> > >>>>>>>>>>>> method
> > >>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
> > >>>>>>>> ProcessorContext.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
> > >>>>> showing:
> > >>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
> > >>>>>>>>>> processorContext);
> > >>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
> > >>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
> > >>>>> VR>
> > >>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
> > >>>>>>>>> RecordContext
> > >>>>>>>>>>>>>>>>> recordContext);
> > >>>>>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
> > >>>>> as
> > >>>>>>>> above
> > >>>>>>>>>> this
> > >>>>>>>>>>>>>>> wouldn't
> > >>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>> sense.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> > >>>>>>> Karimov <
> > >>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > >>>>>>>>>>>> Can't
> > >>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>> pass
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>> as a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
> > >>>>>> processors?
> > >>>>>>>>> Then
> > >>>>>>>>>> we
> > >>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> able
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
> > >>>>> method
> > >>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
> > >>>>>> Matthias
> > >>>>>>>> J.
> > >>>>>>>>>> Sax
> > >>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
> > >>>> call
> > >>>>> to
> > >>>>>>>>>> aggregate
> > >>>>>>>>>>>>>>> anyways.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
> > >>>> Sax
> > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>>>> implement
> > >>>>>>>>>>>> both.
> > >>>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>> allows
> > >>>>>>>>>>>>>>>>>>> us
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>> cast
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
> > >>>>> limit
> > >>>>>>> the
> > >>>>>>>>>>>> visible
> > >>>>>>>>>>>>>>> scope.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
> > >>>> Karimov
> > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
> > >>>>> as
> > >>>>>>> well
> > >>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> > >>>>>>>> Guozhang
> > >>>>>>>>>>>> Wang <
> > >>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
> > >>>> PM,
> > >>>>>>> Jeyhun
> > >>>>>>>>>>>> Karimov
> > >>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> > >>>>>> KAFKA-3907
> > >>>>>>>> [1]
> > >>>>>>>>>> can
> > >>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>> integrated
> > >>>>>>>>>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
> > >>>>>> inside
> > >>>>>>>>>>>>> RecordContext
> > >>>>>>>>>>>>>>>>>>> interface.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
> > >>>> PM
> > >>>>>>>> Matthias
> > >>>>>>>>>> J.
> > >>>>>>>>>>>>> Sax <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to push this
> > >>>>>>> discussion
> > >>>>>>>>>>>> further.
> > >>>>>>>>>>>>> It
> > >>>>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> got
> > >>>>>>>>>>>>>

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

Posted by Guozhang Wang <wa...@gmail.com>.
Thanks for the information Jeyhun. I had also forgot about KAFKA-3907 with
this 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:

1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce
user to consolidate this call as
"processorContext.recordContext().commit()". And internal implementation of
`ProcessorContext.commit()` in `ProcessorContextImpl` is also changed to
this call.

2. Add the `task` reference to the impl class, `ProcessorRecordContext`, so
that it can implement the commit call itself.

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 snippet
below would need to be updated as well.


Guozhang



On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax <ma...@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.
>
>
> -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 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/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. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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 DSL users should not worry about -- and if
> >> one really needs this, an advanced user can still insert a dummy
> >> `transform` to request a commit from there.
> >>
> >> -Matthias
> >>
> >>
> >> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> >>> 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 argument to
> Rich
> >>> methods:
> >>>
> >>> instead of
> >>> public interface RichValueMapper<V, VR, K> {
> >>>     VR apply(final V value,
> >>>              final K key,
> >>>              final RecordContext recordContext);
> >>> }
> >>>
> >>> we can adopt
> >>>
> >>> public interface RichValueMapper<V, VR, K> {
> >>>     VR apply(final V value,
> >>>              final K key,
> >>>              final RecordContext recordContext,
> >>>              final ProcessorContext processorContext);
> >>> }
> >>>
> >>>
> >>> However, in this case, a user can get confused as ProcessorContext and
> >>> RecordContext share some methods with the same name.
> >>>
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>>
> >>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>
> >>>
> >>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> 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 whole task up to this processed record, not only that single
> >> record
> >>>> itself.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je.karimov@gmail.com
> >
> >>>> wrote:
> >>>>
> >>>>> 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<V1, V2, VR, K> {
> >>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>>>> RecordContext
> >>>>>> recordContext);
> >>>>>> }
> >>>>>
> >>>>>
> >>>>>
> >>>>> - Fixed.
> >>>>>
> >>>>>
> >>>>> 2. Some of the listed functions are not necessary since their pairing
> >>>> APIs
> >>>>>> are being deprecated in 1.0 already:
> >>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super
> K,
> >>>> ?
> >>>>>> super V, KR> selector,
> >>>>>>                                    final Serde<KR> keySerde,
> >>>>>>                                    final Serde<V> valSerde);
> >>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>                                  final RichValueJoiner<? super K, ?
> >>>> super
> >>>>>> V,
> >>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>                                  final Serde<K> keySerde,
> >>>>>>                                  final Serde<V> valSerde);
> >>>>>
> >>>>>
> >>>>> -Fixed
> >>>>>
> >>>>> 3. For a few functions where we are adding three APIs for a combo of
> >> both
> >>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>> subtractor,
> >>>>>> I'm wondering if we can just keep one that use "rich" functions for
> >>>> both;
> >>>>>> so that we can have less overloads and let users who only want to
> >>>> access
> >>>>>> one of them to just use dummy parameter declarations. For example:
> >>>>>>
> >>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>> globalKTable,
> >>>>>>                                  final RichKeyValueMapper<? super
> K, ?
> >>>>>> super
> >>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>                                  final RichValueJoiner<? super K, ?
> >>>> super
> >>>>>> V,
> >>>>>> ? super GV, ? extends RV> joiner);
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Agreed. Fixed.
> >>>>>
> >>>>>
> >>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>>>> Initializer also "rich" functions? I.e.
> >>>>>
> >>>>>
> >>>>> - It was a typo. Fixed.
> >>>>>
> >>>>>
> >>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> >>>>>> o.a.k.processor.
> >>>>>>
> >>>>>> 6. I'm not clear why we want to move `commit()` from
> ProcessorContext
> >>>> to
> >>>>>> RecordContext?
> >>>>>>
> >>>>>
> >>>>> -
> >>>>> Because it makes sense logically and  to reduce code maintenance
> (both
> >>>>> interfaces have offset() timestamp() topic() partition() methods),  I
> >>>>> inherit ProcessorContext from RecordContext.
> >>>>> Since we need commit() method both in ProcessorContext and in
> >>>> RecordContext
> >>>>> I move commit() method to parent class (RecordContext).
> >>>>>
> >>>>>
> >>>>> Cheers,
> >>>>> Jeyhun
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> 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<V1, V2, VR, K> {
> >>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>>>> RecordContext
> >>>>>> recordContext);
> >>>>>> }
> >>>>>>
> >>>>>> My motivation is that for lambda expression in J8, users that would
> >> not
> >>>>>> care about the key but only the context, or vice versa, is likely to
> >>>>> write
> >>>>>> it as (value1, value2, dummy, context) -> ... than putting the dummy
> >> at
> >>>>> the
> >>>>>> beginning of the parameter list. Generally speaking we'd like to
> make
> >>>> all
> >>>>>> the "necessary" parameters prior to optional ones.
> >>>>>>
> >>>>>>
> >>>>>> 2. Some of the listed functions are not necessary since their
> pairing
> >>>>> APIs
> >>>>>> are being deprecated in 1.0 already:
> >>>>>>
> >>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super
> K,
> >>>> ?
> >>>>>> super V, KR> selector,
> >>>>>>                                    final Serde<KR> keySerde,
> >>>>>>                                    final Serde<V> valSerde);
> >>>>>>
> >>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>>>                                  final RichValueJoiner<? super K, ?
> >>>> super
> >>>>>> V,
> >>>>>> ? super VT, ? extends VR> joiner,
> >>>>>>                                  final Serde<K> keySerde,
> >>>>>>                                  final Serde<V> valSerde);
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> 3. For a few functions where we are adding three APIs for a combo of
> >>>> both
> >>>>>> mapper / joiner, or both initializer / aggregator, or adder /
> >>>> subtractor,
> >>>>>> I'm wondering if we can just keep one that use "rich" functions for
> >>>> both;
> >>>>>> so that we can have less overloads and let users who only want to
> >>>> access
> >>>>>> one of them to just use dummy parameter declarations. For example:
> >>>>>>
> >>>>>>
> >>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >>>> globalKTable,
> >>>>>>                                  final RichKeyValueMapper<? super
> K, ?
> >>>>>> super
> >>>>>>  V, ? extends GK> keyValueMapper,
> >>>>>>                                  final RichValueJoiner<? super K, ?
> >>>> super
> >>>>>> V,
> >>>>>> ? super GV, ? extends RV> joiner);
> >>>>>>
> >>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR>
> initializer,
> >>>>>>                              final RichAggregator<? super K, ? super
> >> V,
> >>>>> VR>
> >>>>>> aggregator,
> >>>>>>                              final Materialized<K, VR,
> >>>>> KeyValueStore<Bytes,
> >>>>>> byte[]>> materialized);
> >>>>>>
> >>>>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated
> so
> >>>> we
> >>>>> do
> >>>>>> not need to add its rich functions any more.
> >>>>>>
> >>>>>>
> >>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>>>> Initializer also "rich" functions? I.e.
> >>>>>>
> >>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> >>>>>> initializer,
> >>>>>>                                        final RichAggregator<? super
> K,
> >>>> ?
> >>>>>> super V, VR> aggregator);
> >>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> >>>>>> initializer,
> >>>>>>                                        final RichAggregator<? super
> K,
> >>>> ?
> >>>>>> super V, VR> aggregator,
> >>>>>>                                        final Materialized<K, VR,
> >>>>>> WindowStore<Bytes, byte[]>> materialized);
> >>>>>>
> >>>>>>
> >>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> >>>>>> o.a.k.processor.
> >>>>>>
> >>>>>> 6. I'm not clear why we want to move `commit()` from
> ProcessorContext
> >>>> to
> >>>>>> RecordContext? Conceptually I think it would better staying in the
> >>>>>> ProcessorContext. Do you find this not doable in the internal
> >>>>>> implementations?
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com>
> wrote:
> >>>>>>
> >>>>>>>    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 <
> >>>> je.karimov@gmail.com
> >>>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> 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 to KIP.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>                              final Aggregator<? super K, ? super
> >>>> V,
> >>>>>> VR>
> >>>>>>>>> adder,
> >>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
> >>>>>> related
> >>>>>>>> method. So, I had to overload all possible their Rich
> counterparts:
> >>>>>>>>
> >>>>>>>> // adder with non-rich, subtrctor is rich
> >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>                              final Aggregator<? super K, ? super
> V,
> >>>>> VR>
> >>>>>>>> adder,
> >>>>>>>>                              final RichAggregator<? super K, ?
> >>>> super
> >>>>> V,
> >>>>>>> VR>
> >>>>>>>> subtractor,
> >>>>>>>>                              final Materialized<K, VR,
> >>>>>>> KeyValueStore<Bytes,
> >>>>>>>> byte[]>> materialized);
> >>>>>>>>
> >>>>>>>> // adder withrich, subtrctor is non-rich
> >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>                              final RichAggregator<? super K, ?
> >>>> super
> >>>>> V,
> >>>>>>> VR>
> >>>>>>>> adder,
> >>>>>>>>                              final Aggregator<? super K, ? super
> V,
> >>>>> VR>
> >>>>>>>> subtractor,
> >>>>>>>>                              final Materialized<K, VR,
> >>>>>>> KeyValueStore<Bytes,
> >>>>>>>> byte[]>> materialized);
> >>>>>>>>
> >>>>>>>> // both adder and subtractor are rich
> >>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>                              final RichAggregator<? super K, ?
> >>>> super
> >>>>> V,
> >>>>>>> VR>
> >>>>>>>> adder,
> >>>>>>>>                              final RichAggregator<? super K, ?
> >>>> super
> >>>>> V,
> >>>>>>> VR>
> >>>>>>>> subtractor,
> >>>>>>>>                              final Materialized<K, VR,
> >>>>>>> KeyValueStore<Bytes,
> >>>>>>>> byte[]>> materialized);
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>    void commit () {
> >>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>>>> supported
> >>>>>>>> in
> >>>>>>>>> this context");
> >>>>>>>>> Is the exception going to be replaced with real code in the PR ?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> - I added some comments both inside and outside the code snippets
> >>>> in
> >>>>>> KIP.
> >>>>>>>> Specifically, for the code snippet above, we add *commit()* method
> >>>> to
> >>>>>>>> *RecordContext* interface.
> >>>>>>>> However, we want  *commit()* method to be used only for
> >>>>> *RecordContext*
> >>>>>>>> instances (at least for now), so we add
> >>>> UnsupportedOperationException
> >>>>>> in
> >>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
> >>>>>>>> In general, 1) we make RecordContext publicly available within
> >>>>>>>> ProcessorContext,  2) initialize its instance within all required
> >>>>>>>> Processors and 3) pass it as an argument to the related Rich
> >>>>> interfaces
> >>>>>>>> inside Processors.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Jeyhun
> >>>>>>>>
> >>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> >>>> wrote:
> >>>>>>>>
> >>>>>>>>> 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.
> >>>>>>>>>
> >>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>>>                              final Aggregator<? super K, ? super
> >>>> V,
> >>>>>> VR>
> >>>>>>>>> adder,
> >>>>>>>>>
> >>>>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>>>
> >>>>>>>>>   public RecordContext recordContext() {
> >>>>>>>>>     return this.recordContext();
> >>>>>>>>>
> >>>>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>>>
> >>>>>>>>>    void commit () {
> >>>>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>>>> supported
> >>>>>>>> in
> >>>>>>>>> this context");
> >>>>>>>>>
> >>>>>>>>> Is the exception going to be replaced with real code in the PR ?
> >>>>>>>>>
> >>>>>>>>> Cheers
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> >>>>>> je.karimov@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> 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 <
> >>>>>>> je.karimov@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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 <
> >>>>> damian.guy@gmail.com
> >>>>>>>
> >>>>>>>>> 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 Jeyhun Karimov <
> >>>>>> je.karimov@gmail.com
> >>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
> >>>>> not
> >>>>>>>> voted
> >>>>>>>>>> yet
> >>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> >>>>> "atomic"
> >>>>>>>>> merge).
> >>>>>>>>>>>> So
> >>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> >>>> some
> >>>>>>> minor
> >>>>>>>>>>>> updates
> >>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
> >>>> if
> >>>>> I
> >>>>>> am
> >>>>>>>>> wrong
> >>>>>>>>>>>> or I
> >>>>>>>>>>>>> misunderstood.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> >>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> >>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>> 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
> >>>>> would
> >>>>>>>> still
> >>>>>>>>>>>> need to
> >>>>>>>>>>>>>>> update your proposal for people to comment and vote
> >>>> on.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> >>>>>>> yuzhihong@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> One interface is cleaner.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> >>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> >>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>> interfaces
> >>>>>>>>>>>>>>>> into 1
> >>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> >>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 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
> >>>> 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
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> would not be a problem.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Our initial idea was similar to :
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> >>>>>>>>> implements
> >>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> >>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> >>>>>> RichXXX
> >>>>>>>> or
> >>>>>>>>>>>>>> XXXWithKey
> >>>>>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>>> the called method and continue accordingly.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
> >>>> to
> >>>>>>> revert
> >>>>>>>>> the
> >>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>> to this again.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >>>>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Jeyhun,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> >>>> was
> >>>>>>>>> thinking
> >>>>>>>>>>>> more
> >>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> >>>> RichXXXX
> >>>>>> and
> >>>>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>>>> 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?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>>>> confluence/pages/viewpage.
> >>>>>>>>>>>>>>>>>> action?pageId=73637757
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> >>>>> Karimov <
> >>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> >>>>> issue.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> >>>>> Guy
> >>>>>> <
> >>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>> overrides.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> >>>>> RecordContext,
> >>>>>>> you
> >>>>>>>>> are
> >>>>>>>>>>>>> right.
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> >>>> the
> >>>>>>> full
> >>>>>>>>>>>> picture
> >>>>>>>>>>>>> as
> >>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> parts
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> >>>> as
> >>>>> I
> >>>>>>>>> thought.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> >>>> Damian
> >>>>>> Guy
> >>>>>>> <
> >>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
> >>>>> to
> >>>>>>>> cast a
> >>>>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
> >>>>>>>>>>>>>> InternalProcessorContext
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> init()
> >>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
> >>>>> and i
> >>>>>>>> think
> >>>>>>>>>> it
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>> remain
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
> >>>>>>>> recordContext()
> >>>>>>>>>>>> method
> >>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
> >>>>>>>> ProcessorContext.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
> >>>>> showing:
> >>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
> >>>>>>>>>> processorContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
> >>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
> >>>>> VR>
> >>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
> >>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
> >>>>> as
> >>>>>>>> above
> >>>>>>>>>> this
> >>>>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>> sense.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> >>>>>>> Karimov <
> >>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>> Can't
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>> pass
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
> >>>>>> processors?
> >>>>>>>>> Then
> >>>>>>>>>> we
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> able
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
> >>>>> method
> >>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
> >>>>>> Matthias
> >>>>>>>> J.
> >>>>>>>>>> Sax
> >>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
> >>>> call
> >>>>> to
> >>>>>>>>>> aggregate
> >>>>>>>>>>>>>>> anyways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
> >>>> Sax
> >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>> implement
> >>>>>>>>>>>> both.
> >>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>> allows
> >>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> cast
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
> >>>>> limit
> >>>>>>> the
> >>>>>>>>>>>> visible
> >>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
> >>>> Karimov
> >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
> >>>>> as
> >>>>>>> well
> >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> >>>>>>>> Guozhang
> >>>>>>>>>>>> Wang <
> >>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
> >>>> PM,
> >>>>>>> Jeyhun
> >>>>>>>>>>>> Karimov
> >>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>> KAFKA-3907
> >>>>>>>> [1]
> >>>>>>>>>> can
> >>>>>>>>>>>>> also
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>> integrated
> >>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
> >>>>>> inside
> >>>>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
> >>>> PM
> >>>>>>>> Matthias
> >>>>>>>>>> J.
> >>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>> information
> >>>>>>>>>>>>> only.
> >>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stateful computation you
> >>>> should
> >>>>>>>> rather
> >>>>>>>>>> use
> >>>>>>>>>>>>>>>>> #transform().
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, as pointed out,
> >>>> we
> >>>>>>> would
> >>>>>>>>>> need
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> switch
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> supplier-pattern introducing
> >>>>> many
> >>>>>>>> more
> >>>>>>>>>>>>>> overloads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For those reason, I advocate
> >>>>> for
> >>>>>> a
> >>>>>>>>> simple
> >>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>> with a
> >>>>>>>>>>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that passes in a
> >>>> RecordContext
> >>>>>>>> object.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
> >>>>> Wang
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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, so I
> >>>>> will
> >>>>>>> not
> >>>>>>>>>>>> mention
> >>>>>>>>>>>>>>> them.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The related discussions are
> >>>>>>> spread
> >>>>>>>> in
> >>>>>>>>>>>>> KIP-149
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KIP-159)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion threads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. The idea of rich
> >>>> functions
> >>>>>>> came
> >>>>>>>>> into
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> stage
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP-149,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion thread. As a
> >>>>> result
> >>>>>> we
> >>>>>>>>>>>> extended
> >>>>>>>>>>>>>>> KIP-149
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> support
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Rich
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> functions as well.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.  To as part of the Rich
> >>>>>>>> functions,
> >>>>>>>>>> we
> >>>>>>>>>>>>>>> provided
> >>>>>>>>>>>>>>>>> init
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (ProcessorContext)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method. Afterwards, Dammian
> >>>>>>>> suggested
> >>>>>>>>>>>> that
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext to users.
> >>>>> As a
> >>>>>>>>> result,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> separated
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two separate KIPs, as it
> >>>>> seems
> >>>>>>> they
> >>>>>>>>> can
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> solved
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> parallel.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - One approach we
> >>>> considered
> >>>>>> was
> >>>>>>> :
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
> >>>>>>>>> ValueMapperWithKey<K,
> >>>>>>>>>> V,
> >>>>>>>>>>>
> >
>
>


-- 
-- Guozhang

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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.


-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 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/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. Sax <ma...@confluent.io>
> wrote:
> 
>> 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 DSL users should not worry about -- and if
>> one really needs this, an advanced user can still insert a dummy
>> `transform` to request a commit from there.
>>
>> -Matthias
>>
>>
>> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
>>> 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 argument to Rich
>>> methods:
>>>
>>> instead of
>>> public interface RichValueMapper<V, VR, K> {
>>>     VR apply(final V value,
>>>              final K key,
>>>              final RecordContext recordContext);
>>> }
>>>
>>> we can adopt
>>>
>>> public interface RichValueMapper<V, VR, K> {
>>>     VR apply(final V value,
>>>              final K key,
>>>              final RecordContext recordContext,
>>>              final ProcessorContext processorContext);
>>> }
>>>
>>>
>>> However, in this case, a user can get confused as ProcessorContext and
>>> RecordContext share some methods with the same name.
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>>
>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>
>>>
>>> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>
>>>> 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 whole task up to this processed record, not only that single
>> record
>>>> itself.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je...@gmail.com>
>>>> wrote:
>>>>
>>>>> 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<V1, V2, VR, K> {
>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>> RecordContext
>>>>>> recordContext);
>>>>>> }
>>>>>
>>>>>
>>>>>
>>>>> - Fixed.
>>>>>
>>>>>
>>>>> 2. Some of the listed functions are not necessary since their pairing
>>>> APIs
>>>>>> are being deprecated in 1.0 already:
>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
>>>> ?
>>>>>> super V, KR> selector,
>>>>>>                                    final Serde<KR> keySerde,
>>>>>>                                    final Serde<V> valSerde);
>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>                                  final RichValueJoiner<? super K, ?
>>>> super
>>>>>> V,
>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>                                  final Serde<K> keySerde,
>>>>>>                                  final Serde<V> valSerde);
>>>>>
>>>>>
>>>>> -Fixed
>>>>>
>>>>> 3. For a few functions where we are adding three APIs for a combo of
>> both
>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>> subtractor,
>>>>>> I'm wondering if we can just keep one that use "rich" functions for
>>>> both;
>>>>>> so that we can have less overloads and let users who only want to
>>>> access
>>>>>> one of them to just use dummy parameter declarations. For example:
>>>>>>
>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>> globalKTable,
>>>>>>                                  final RichKeyValueMapper<? super K, ?
>>>>>> super
>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>                                  final RichValueJoiner<? super K, ?
>>>> super
>>>>>> V,
>>>>>> ? super GV, ? extends RV> joiner);
>>>>>
>>>>>
>>>>>
>>>>> -Agreed. Fixed.
>>>>>
>>>>>
>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>> Initializer also "rich" functions? I.e.
>>>>>
>>>>>
>>>>> - It was a typo. Fixed.
>>>>>
>>>>>
>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
>>>>>> o.a.k.processor.
>>>>>>
>>>>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
>>>> to
>>>>>> RecordContext?
>>>>>>
>>>>>
>>>>> -
>>>>> Because it makes sense logically and  to reduce code maintenance (both
>>>>> interfaces have offset() timestamp() topic() partition() methods),  I
>>>>> inherit ProcessorContext from RecordContext.
>>>>> Since we need commit() method both in ProcessorContext and in
>>>> RecordContext
>>>>> I move commit() method to parent class (RecordContext).
>>>>>
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> 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<V1, V2, VR, K> {
>>>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>>>> RecordContext
>>>>>> recordContext);
>>>>>> }
>>>>>>
>>>>>> My motivation is that for lambda expression in J8, users that would
>> not
>>>>>> care about the key but only the context, or vice versa, is likely to
>>>>> write
>>>>>> it as (value1, value2, dummy, context) -> ... than putting the dummy
>> at
>>>>> the
>>>>>> beginning of the parameter list. Generally speaking we'd like to make
>>>> all
>>>>>> the "necessary" parameters prior to optional ones.
>>>>>>
>>>>>>
>>>>>> 2. Some of the listed functions are not necessary since their pairing
>>>>> APIs
>>>>>> are being deprecated in 1.0 already:
>>>>>>
>>>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
>>>> ?
>>>>>> super V, KR> selector,
>>>>>>                                    final Serde<KR> keySerde,
>>>>>>                                    final Serde<V> valSerde);
>>>>>>
>>>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>>>                                  final RichValueJoiner<? super K, ?
>>>> super
>>>>>> V,
>>>>>> ? super VT, ? extends VR> joiner,
>>>>>>                                  final Serde<K> keySerde,
>>>>>>                                  final Serde<V> valSerde);
>>>>>>
>>>>>>
>>>>>>
>>>>>> 3. For a few functions where we are adding three APIs for a combo of
>>>> both
>>>>>> mapper / joiner, or both initializer / aggregator, or adder /
>>>> subtractor,
>>>>>> I'm wondering if we can just keep one that use "rich" functions for
>>>> both;
>>>>>> so that we can have less overloads and let users who only want to
>>>> access
>>>>>> one of them to just use dummy parameter declarations. For example:
>>>>>>
>>>>>>
>>>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>>>> globalKTable,
>>>>>>                                  final RichKeyValueMapper<? super K, ?
>>>>>> super
>>>>>>  V, ? extends GK> keyValueMapper,
>>>>>>                                  final RichValueJoiner<? super K, ?
>>>> super
>>>>>> V,
>>>>>> ? super GV, ? extends RV> joiner);
>>>>>>
>>>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
>>>>>>                              final RichAggregator<? super K, ? super
>> V,
>>>>> VR>
>>>>>> aggregator,
>>>>>>                              final Materialized<K, VR,
>>>>> KeyValueStore<Bytes,
>>>>>> byte[]>> materialized);
>>>>>>
>>>>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated so
>>>> we
>>>>> do
>>>>>> not need to add its rich functions any more.
>>>>>>
>>>>>>
>>>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>>>> Initializer also "rich" functions? I.e.
>>>>>>
>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
>>>>>> initializer,
>>>>>>                                        final RichAggregator<? super K,
>>>> ?
>>>>>> super V, VR> aggregator);
>>>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
>>>>>> initializer,
>>>>>>                                        final RichAggregator<? super K,
>>>> ?
>>>>>> super V, VR> aggregator,
>>>>>>                                        final Materialized<K, VR,
>>>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>>>
>>>>>>
>>>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
>>>>>> o.a.k.processor.
>>>>>>
>>>>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
>>>> to
>>>>>> RecordContext? Conceptually I think it would better staying in the
>>>>>> ProcessorContext. Do you find this not doable in the internal
>>>>>> implementations?
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
>>>>>>
>>>>>>>    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 <
>>>> je.karimov@gmail.com
>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 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 to KIP.
>>>>>>>>
>>>>>>>>
>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>                              final Aggregator<? super K, ? super
>>>> V,
>>>>>> VR>
>>>>>>>>> adder,
>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
>>>>>> related
>>>>>>>> method. So, I had to overload all possible their Rich counterparts:
>>>>>>>>
>>>>>>>> // adder with non-rich, subtrctor is rich
>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>                              final Aggregator<? super K, ? super V,
>>>>> VR>
>>>>>>>> adder,
>>>>>>>>                              final RichAggregator<? super K, ?
>>>> super
>>>>> V,
>>>>>>> VR>
>>>>>>>> subtractor,
>>>>>>>>                              final Materialized<K, VR,
>>>>>>> KeyValueStore<Bytes,
>>>>>>>> byte[]>> materialized);
>>>>>>>>
>>>>>>>> // adder withrich, subtrctor is non-rich
>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>                              final RichAggregator<? super K, ?
>>>> super
>>>>> V,
>>>>>>> VR>
>>>>>>>> adder,
>>>>>>>>                              final Aggregator<? super K, ? super V,
>>>>> VR>
>>>>>>>> subtractor,
>>>>>>>>                              final Materialized<K, VR,
>>>>>>> KeyValueStore<Bytes,
>>>>>>>> byte[]>> materialized);
>>>>>>>>
>>>>>>>> // both adder and subtractor are rich
>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>                              final RichAggregator<? super K, ?
>>>> super
>>>>> V,
>>>>>>> VR>
>>>>>>>> adder,
>>>>>>>>                              final RichAggregator<? super K, ?
>>>> super
>>>>> V,
>>>>>>> VR>
>>>>>>>> subtractor,
>>>>>>>>                              final Materialized<K, VR,
>>>>>>> KeyValueStore<Bytes,
>>>>>>>> byte[]>> materialized);
>>>>>>>>
>>>>>>>>
>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>    void commit () {
>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>> supported
>>>>>>>> in
>>>>>>>>> this context");
>>>>>>>>> Is the exception going to be replaced with real code in the PR ?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> - I added some comments both inside and outside the code snippets
>>>> in
>>>>>> KIP.
>>>>>>>> Specifically, for the code snippet above, we add *commit()* method
>>>> to
>>>>>>>> *RecordContext* interface.
>>>>>>>> However, we want  *commit()* method to be used only for
>>>>> *RecordContext*
>>>>>>>> instances (at least for now), so we add
>>>> UnsupportedOperationException
>>>>>> in
>>>>>>>> all classes/interfaces that extend/implement *RecordContext.*
>>>>>>>> In general, 1) we make RecordContext publicly available within
>>>>>>>> ProcessorContext,  2) initialize its instance within all required
>>>>>>>> Processors and 3) pass it as an argument to the related Rich
>>>>> interfaces
>>>>>>>> inside Processors.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
>>>> wrote:
>>>>>>>>
>>>>>>>>> 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.
>>>>>>>>>
>>>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>>>                              final Aggregator<? super K, ? super
>>>> V,
>>>>>> VR>
>>>>>>>>> adder,
>>>>>>>>>
>>>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>>>
>>>>>>>>>   public RecordContext recordContext() {
>>>>>>>>>     return this.recordContext();
>>>>>>>>>
>>>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>>>
>>>>>>>>>    void commit () {
>>>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>>>> supported
>>>>>>>> in
>>>>>>>>> this context");
>>>>>>>>>
>>>>>>>>> Is the exception going to be replaced with real code in the PR ?
>>>>>>>>>
>>>>>>>>> Cheers
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>>>> je.karimov@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 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 <
>>>>>>> je.karimov@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 <
>>>>> damian.guy@gmail.com
>>>>>>>
>>>>>>>>> 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 Jeyhun Karimov <
>>>>>> je.karimov@gmail.com
>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
>>>>> not
>>>>>>>> voted
>>>>>>>>>> yet
>>>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
>>>>> "atomic"
>>>>>>>>> merge).
>>>>>>>>>>>> So
>>>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
>>>> some
>>>>>>> minor
>>>>>>>>>>>> updates
>>>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
>>>> if
>>>>> I
>>>>>> am
>>>>>>>>> wrong
>>>>>>>>>>>> or I
>>>>>>>>>>>>> misunderstood.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>> 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
>>>>> would
>>>>>>>> still
>>>>>>>>>>>> need to
>>>>>>>>>>>>>>> update your proposal for people to comment and vote
>>>> on.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>>>> yuzhihong@gmail.com
>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>> interfaces
>>>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 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
>>>> 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
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>>>> implements
>>>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>>>> RichXXX
>>>>>>>> or
>>>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>>>> to
>>>>>>> revert
>>>>>>>>> the
>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>>>> was
>>>>>>>>> thinking
>>>>>>>>>>>> more
>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>>>> RichXXXX
>>>>>> and
>>>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>>>> Karimov <
>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
>>>>> issue.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
>>>>> Guy
>>>>>> <
>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>> overrides.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
>>>>> RecordContext,
>>>>>>> you
>>>>>>>>> are
>>>>>>>>>>>>> right.
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
>>>> the
>>>>>>> full
>>>>>>>>>>>> picture
>>>>>>>>>>>>> as
>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> parts
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
>>>> as
>>>>> I
>>>>>>>>> thought.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
>>>> Damian
>>>>>> Guy
>>>>>>> <
>>>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
>>>>> to
>>>>>>>> cast a
>>>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
>>>>>>>>>>>>>> InternalProcessorContext
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> init()
>>>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
>>>>> and i
>>>>>>>> think
>>>>>>>>>> it
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>> remain
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
>>>>>>>> recordContext()
>>>>>>>>>>>> method
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
>>>>>>>> ProcessorContext.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
>>>>> showing:
>>>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
>>>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
>>>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
>>>>> VR>
>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
>>>>> as
>>>>>>>> above
>>>>>>>>>> this
>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>> sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
>>>>>>> Karimov <
>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>> Can't
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
>>>>>> processors?
>>>>>>>>> Then
>>>>>>>>>> we
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> able
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
>>>>> method
>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
>>>>>> Matthias
>>>>>>>> J.
>>>>>>>>>> Sax
>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
>>>> call
>>>>> to
>>>>>>>>>> aggregate
>>>>>>>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
>>>> Sax
>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>> implement
>>>>>>>>>>>> both.
>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>> allows
>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> cast
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
>>>>> limit
>>>>>>> the
>>>>>>>>>>>> visible
>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
>>>> Karimov
>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
>>>>> as
>>>>>>> well
>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
>>>>>>>> Guozhang
>>>>>>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
>>>> PM,
>>>>>>> Jeyhun
>>>>>>>>>>>> Karimov
>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>> KAFKA-3907
>>>>>>>> [1]
>>>>>>>>>> can
>>>>>>>>>>>>> also
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> integrated
>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
>>>>>> inside
>>>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
>>>> PM
>>>>>>>> Matthias
>>>>>>>>>> J.
>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>> information
>>>>>>>>>>>>> only.
>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stateful computation you
>>>> should
>>>>>>>> rather
>>>>>>>>>> use
>>>>>>>>>>>>>>>>> #transform().
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, as pointed out,
>>>> we
>>>>>>> would
>>>>>>>>>> need
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> switch
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> supplier-pattern introducing
>>>>> many
>>>>>>>> more
>>>>>>>>>>>>>> overloads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For those reason, I advocate
>>>>> for
>>>>>> a
>>>>>>>>> simple
>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>> with a
>>>>>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that passes in a
>>>> RecordContext
>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
>>>>> Wang
>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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, so I
>>>>> will
>>>>>>> not
>>>>>>>>>>>> mention
>>>>>>>>>>>>>>> them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The related discussions are
>>>>>>> spread
>>>>>>>> in
>>>>>>>>>>>>> KIP-149
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KIP-159)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion threads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. The idea of rich
>>>> functions
>>>>>>> came
>>>>>>>>> into
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> stage
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>> KIP-149,
>>>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion thread. As a
>>>>> result
>>>>>> we
>>>>>>>>>>>> extended
>>>>>>>>>>>>>>> KIP-149
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> functions as well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.  To as part of the Rich
>>>>>>>> functions,
>>>>>>>>>> we
>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>> init
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (ProcessorContext)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method. Afterwards, Dammian
>>>>>>>> suggested
>>>>>>>>>>>> that
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext to users.
>>>>> As a
>>>>>>>>> result,
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> separated
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two separate KIPs, as it
>>>>> seems
>>>>>>> they
>>>>>>>>> can
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> solved
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> parallel.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - One approach we
>>>> considered
>>>>>> was
>>>>>>> :
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
>>>>>>>>> ValueMapperWithKey<K,
>>>>>>>>>> V,
>>>>>>>>>>>
> 


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

Posted by Jeyhun Karimov <je...@gmail.com>.
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
[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. Sax <ma...@confluent.io>
wrote:

> 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 DSL users should not worry about -- and if
> one really needs this, an advanced user can still insert a dummy
> `transform` to request a commit from there.
>
> -Matthias
>
>
> On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> > 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 argument to Rich
> > methods:
> >
> > instead of
> > public interface RichValueMapper<V, VR, K> {
> >     VR apply(final V value,
> >              final K key,
> >              final RecordContext recordContext);
> > }
> >
> > we can adopt
> >
> > public interface RichValueMapper<V, VR, K> {
> >     VR apply(final V value,
> >              final K key,
> >              final RecordContext recordContext,
> >              final ProcessorContext processorContext);
> > }
> >
> >
> > However, in this case, a user can get confused as ProcessorContext and
> > RecordContext share some methods with the same name.
> >
> >
> > Cheers,
> > Jeyhun
> >
> >
> > [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >
> >
> > On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> >> 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 whole task up to this processed record, not only that single
> record
> >> itself.
> >>
> >>
> >> Guozhang
> >>
> >> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je...@gmail.com>
> >> wrote:
> >>
> >>> 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<V1, V2, VR, K> {
> >>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>> RecordContext
> >>>> recordContext);
> >>>> }
> >>>
> >>>
> >>>
> >>> - Fixed.
> >>>
> >>>
> >>> 2. Some of the listed functions are not necessary since their pairing
> >> APIs
> >>>> are being deprecated in 1.0 already:
> >>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
> >> ?
> >>>> super V, KR> selector,
> >>>>                                    final Serde<KR> keySerde,
> >>>>                                    final Serde<V> valSerde);
> >>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>                                  final RichValueJoiner<? super K, ?
> >> super
> >>>> V,
> >>>> ? super VT, ? extends VR> joiner,
> >>>>                                  final Serde<K> keySerde,
> >>>>                                  final Serde<V> valSerde);
> >>>
> >>>
> >>> -Fixed
> >>>
> >>> 3. For a few functions where we are adding three APIs for a combo of
> both
> >>>> mapper / joiner, or both initializer / aggregator, or adder /
> >> subtractor,
> >>>> I'm wondering if we can just keep one that use "rich" functions for
> >> both;
> >>>> so that we can have less overloads and let users who only want to
> >> access
> >>>> one of them to just use dummy parameter declarations. For example:
> >>>>
> >>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >> globalKTable,
> >>>>                                  final RichKeyValueMapper<? super K, ?
> >>>> super
> >>>>  V, ? extends GK> keyValueMapper,
> >>>>                                  final RichValueJoiner<? super K, ?
> >> super
> >>>> V,
> >>>> ? super GV, ? extends RV> joiner);
> >>>
> >>>
> >>>
> >>> -Agreed. Fixed.
> >>>
> >>>
> >>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>> Initializer also "rich" functions? I.e.
> >>>
> >>>
> >>> - It was a typo. Fixed.
> >>>
> >>>
> >>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> >>>> o.a.k.processor.
> >>>>
> >>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
> >> to
> >>>> RecordContext?
> >>>>
> >>>
> >>> -
> >>> Because it makes sense logically and  to reduce code maintenance (both
> >>> interfaces have offset() timestamp() topic() partition() methods),  I
> >>> inherit ProcessorContext from RecordContext.
> >>> Since we need commit() method both in ProcessorContext and in
> >> RecordContext
> >>> I move commit() method to parent class (RecordContext).
> >>>
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>>
> >>>
> >>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
> >>> wrote:
> >>>
> >>>> 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<V1, V2, VR, K> {
> >>>>     VR apply(final V1 value1, final V2 value2, final K key, final
> >>>> RecordContext
> >>>> recordContext);
> >>>> }
> >>>>
> >>>> My motivation is that for lambda expression in J8, users that would
> not
> >>>> care about the key but only the context, or vice versa, is likely to
> >>> write
> >>>> it as (value1, value2, dummy, context) -> ... than putting the dummy
> at
> >>> the
> >>>> beginning of the parameter list. Generally speaking we'd like to make
> >> all
> >>>> the "necessary" parameters prior to optional ones.
> >>>>
> >>>>
> >>>> 2. Some of the listed functions are not necessary since their pairing
> >>> APIs
> >>>> are being deprecated in 1.0 already:
> >>>>
> >>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
> >> ?
> >>>> super V, KR> selector,
> >>>>                                    final Serde<KR> keySerde,
> >>>>                                    final Serde<V> valSerde);
> >>>>
> >>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >>>>                                  final RichValueJoiner<? super K, ?
> >> super
> >>>> V,
> >>>> ? super VT, ? extends VR> joiner,
> >>>>                                  final Serde<K> keySerde,
> >>>>                                  final Serde<V> valSerde);
> >>>>
> >>>>
> >>>>
> >>>> 3. For a few functions where we are adding three APIs for a combo of
> >> both
> >>>> mapper / joiner, or both initializer / aggregator, or adder /
> >> subtractor,
> >>>> I'm wondering if we can just keep one that use "rich" functions for
> >> both;
> >>>> so that we can have less overloads and let users who only want to
> >> access
> >>>> one of them to just use dummy parameter declarations. For example:
> >>>>
> >>>>
> >>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> >> globalKTable,
> >>>>                                  final RichKeyValueMapper<? super K, ?
> >>>> super
> >>>>  V, ? extends GK> keyValueMapper,
> >>>>                                  final RichValueJoiner<? super K, ?
> >> super
> >>>> V,
> >>>> ? super GV, ? extends RV> joiner);
> >>>>
> >>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
> >>>>                              final RichAggregator<? super K, ? super
> V,
> >>> VR>
> >>>> aggregator,
> >>>>                              final Materialized<K, VR,
> >>> KeyValueStore<Bytes,
> >>>> byte[]>> materialized);
> >>>>
> >>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated so
> >> we
> >>> do
> >>>> not need to add its rich functions any more.
> >>>>
> >>>>
> >>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> >>>> Initializer also "rich" functions? I.e.
> >>>>
> >>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> >>>> initializer,
> >>>>                                        final RichAggregator<? super K,
> >> ?
> >>>> super V, VR> aggregator);
> >>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> >>>> initializer,
> >>>>                                        final RichAggregator<? super K,
> >> ?
> >>>> super V, VR> aggregator,
> >>>>                                        final Materialized<K, VR,
> >>>> WindowStore<Bytes, byte[]>> materialized);
> >>>>
> >>>>
> >>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> >>>> o.a.k.processor.
> >>>>
> >>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
> >> to
> >>>> RecordContext? Conceptually I think it would better staying in the
> >>>> ProcessorContext. Do you find this not doable in the internal
> >>>> implementations?
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>>
> >>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
> >>>>
> >>>>>    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 <
> >> je.karimov@gmail.com
> >>>>
> >>>>> wrote:
> >>>>>
> >>>>>> 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 to KIP.
> >>>>>>
> >>>>>>
> >>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>                              final Aggregator<? super K, ? super
> >> V,
> >>>> VR>
> >>>>>>> adder,
> >>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
> >>>> related
> >>>>>> method. So, I had to overload all possible their Rich counterparts:
> >>>>>>
> >>>>>> // adder with non-rich, subtrctor is rich
> >>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>                              final Aggregator<? super K, ? super V,
> >>> VR>
> >>>>>> adder,
> >>>>>>                              final RichAggregator<? super K, ?
> >> super
> >>> V,
> >>>>> VR>
> >>>>>> subtractor,
> >>>>>>                              final Materialized<K, VR,
> >>>>> KeyValueStore<Bytes,
> >>>>>> byte[]>> materialized);
> >>>>>>
> >>>>>> // adder withrich, subtrctor is non-rich
> >>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>                              final RichAggregator<? super K, ?
> >> super
> >>> V,
> >>>>> VR>
> >>>>>> adder,
> >>>>>>                              final Aggregator<? super K, ? super V,
> >>> VR>
> >>>>>> subtractor,
> >>>>>>                              final Materialized<K, VR,
> >>>>> KeyValueStore<Bytes,
> >>>>>> byte[]>> materialized);
> >>>>>>
> >>>>>> // both adder and subtractor are rich
> >>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>                              final RichAggregator<? super K, ?
> >> super
> >>> V,
> >>>>> VR>
> >>>>>> adder,
> >>>>>>                              final RichAggregator<? super K, ?
> >> super
> >>> V,
> >>>>> VR>
> >>>>>> subtractor,
> >>>>>>                              final Materialized<K, VR,
> >>>>> KeyValueStore<Bytes,
> >>>>>> byte[]>> materialized);
> >>>>>>
> >>>>>>
> >>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>    void commit () {
> >>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>> supported
> >>>>>> in
> >>>>>>> this context");
> >>>>>>> Is the exception going to be replaced with real code in the PR ?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> - I added some comments both inside and outside the code snippets
> >> in
> >>>> KIP.
> >>>>>> Specifically, for the code snippet above, we add *commit()* method
> >> to
> >>>>>> *RecordContext* interface.
> >>>>>> However, we want  *commit()* method to be used only for
> >>> *RecordContext*
> >>>>>> instances (at least for now), so we add
> >> UnsupportedOperationException
> >>>> in
> >>>>>> all classes/interfaces that extend/implement *RecordContext.*
> >>>>>> In general, 1) we make RecordContext publicly available within
> >>>>>> ProcessorContext,  2) initialize its instance within all required
> >>>>>> Processors and 3) pass it as an argument to the related Rich
> >>> interfaces
> >>>>>> inside Processors.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Jeyhun
> >>>>>>
> >>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> >> wrote:
> >>>>>>
> >>>>>>> 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.
> >>>>>>>
> >>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >>>>>>>                              final Aggregator<? super K, ? super
> >> V,
> >>>> VR>
> >>>>>>> adder,
> >>>>>>>
> >>>>>>> The adder doesn't need to be RichAggregator ?
> >>>>>>>
> >>>>>>>   public RecordContext recordContext() {
> >>>>>>>     return this.recordContext();
> >>>>>>>
> >>>>>>> Can you explain a bit about the above implementation ?
> >>>>>>>
> >>>>>>>    void commit () {
> >>>>>>>      throw new UnsupportedOperationException("commit() is not
> >>>>> supported
> >>>>>> in
> >>>>>>> this context");
> >>>>>>>
> >>>>>>> Is the exception going to be replaced with real code in the PR ?
> >>>>>>>
> >>>>>>> Cheers
> >>>>>>>
> >>>>>>>
> >>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> >>>> je.karimov@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> 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 <
> >>>>> je.karimov@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> 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 <
> >>> damian.guy@gmail.com
> >>>>>
> >>>>>>> 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 Jeyhun Karimov <
> >>>> je.karimov@gmail.com
> >>>>>>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
> >>> not
> >>>>>> voted
> >>>>>>>> yet
> >>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
> >>> "atomic"
> >>>>>>> merge).
> >>>>>>>>>> So
> >>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
> >> some
> >>>>> minor
> >>>>>>>>>> updates
> >>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
> >> if
> >>> I
> >>>> am
> >>>>>>> wrong
> >>>>>>>>>> or I
> >>>>>>>>>>> misunderstood.
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Jeyhun
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> >>>>> damian.guy@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> +1
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> >>>>> wangguoz@gmail.com>
> >>>>>>>>>> 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
> >>> would
> >>>>>> still
> >>>>>>>>>> need to
> >>>>>>>>>>>>> update your proposal for people to comment and vote
> >> on.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> >>>>> yuzhihong@gmail.com
> >>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> One interface is cleaner.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> >>>>>>> bbejeck@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
> >>>>> ValueXXXXWithKey
> >>>>>>>>>>> interfaces
> >>>>>>>>>>>>>> into 1
> >>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> >>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 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
> >> 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
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> would not be a problem.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Our initial idea was similar to :
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> >>>>>>> implements
> >>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
> >>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> So, we check the type of object, whether it is
> >>>> RichXXX
> >>>>>> or
> >>>>>>>>>>>> XXXWithKey
> >>>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>> the called method and continue accordingly.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If this is ok with the community, I would like
> >> to
> >>>>> revert
> >>>>>>> the
> >>>>>>>>>>>> current
> >>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>> to this again.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >>>>>>>>>> damian.guy@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Jeyhun,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
> >> was
> >>>>>>> thinking
> >>>>>>>>>> more
> >>>>>>>>>>>>> along
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
> >> RichXXXX
> >>>> and
> >>>>>>>>>>>>> ValueXXXXWithKey
> >>>>>>>>>>>>>>> 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?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 free to comment on this.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>> confluence/pages/viewpage.
> >>>>>>>>>>>>>>>> action?pageId=73637757
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> >>> Karimov <
> >>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
> >>> issue.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
> >>> Guy
> >>>> <
> >>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>> overrides.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
> >>> RecordContext,
> >>>>> you
> >>>>>>> are
> >>>>>>>>>>> right.
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
> >> the
> >>>>> full
> >>>>>>>>>> picture
> >>>>>>>>>>> as
> >>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>> parts
> >>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
> >> as
> >>> I
> >>>>>>> thought.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
> >> Damian
> >>>> Guy
> >>>>> <
> >>>>>>>>>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
> >>> to
> >>>>>> cast a
> >>>>>>>>>>>>>>> ProcessorContext
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
> >>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
> >>>>>>>>>>>> InternalProcessorContext
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> init()
> >>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
> >>> and i
> >>>>>> think
> >>>>>>>> it
> >>>>>>>>>>>> should
> >>>>>>>>>>>>>>> remain
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
> >>>>>> recordContext()
> >>>>>>>>>> method
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
> >>>>>> ProcessorContext.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
> >>> showing:
> >>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
> >>>>>>>> processorContext);
> >>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
> >>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
> >>> VR>
> >>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
> >>>>>>> RecordContext
> >>>>>>>>>>>>>>> recordContext);
> >>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
> >>> as
> >>>>>> above
> >>>>>>>> this
> >>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>> sense.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> >>>>> Karimov <
> >>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>> Can't
> >>>>>>>>>>> we
> >>>>>>>>>>>>> pass
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
> >>>> processors?
> >>>>>>> Then
> >>>>>>>> we
> >>>>>>>>>>>> would
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> able
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
> >>> method
> >>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
> >>>> Matthias
> >>>>>> J.
> >>>>>>>> Sax
> >>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
> >> call
> >>> to
> >>>>>>>> aggregate
> >>>>>>>>>>>>> anyways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
> >> Sax
> >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>> implement
> >>>>>>>>>> both.
> >>>>>>>>>>>> This
> >>>>>>>>>>>>>>> allows
> >>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> cast
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
> >>> limit
> >>>>> the
> >>>>>>>>>> visible
> >>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
> >> Karimov
> >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
> >>> as
> >>>>> well
> >>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> >>>>>> Guozhang
> >>>>>>>>>> Wang <
> >>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
> >> PM,
> >>>>> Jeyhun
> >>>>>>>>>> Karimov
> >>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>> KAFKA-3907
> >>>>>> [1]
> >>>>>>>> can
> >>>>>>>>>>> also
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> integrated
> >>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
> >>>> inside
> >>>>>>>>>>> RecordContext
> >>>>>>>>>>>>>>>>> interface.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
> >> PM
> >>>>>> Matthias
> >>>>>>>> J.
> >>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>> information
> >>>>>>>>>>> only.
> >>>>>>>>>>>>> If
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stateful computation you
> >> should
> >>>>>> rather
> >>>>>>>> use
> >>>>>>>>>>>>>>> #transform().
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, as pointed out,
> >> we
> >>>>> would
> >>>>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>>>> switch
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> supplier-pattern introducing
> >>> many
> >>>>>> more
> >>>>>>>>>>>> overloads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For those reason, I advocate
> >>> for
> >>>> a
> >>>>>>> simple
> >>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>> with a
> >>>>>>>>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that passes in a
> >> RecordContext
> >>>>>> object.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
> >>> Wang
> >>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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, so I
> >>> will
> >>>>> not
> >>>>>>>>>> mention
> >>>>>>>>>>>>> them.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The related discussions are
> >>>>> spread
> >>>>>> in
> >>>>>>>>>>> KIP-149
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KIP-159)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion threads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. The idea of rich
> >> functions
> >>>>> came
> >>>>>>> into
> >>>>>>>>>> the
> >>>>>>>>>>>>> stage
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>> KIP-149,
> >>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion thread. As a
> >>> result
> >>>> we
> >>>>>>>>>> extended
> >>>>>>>>>>>>> KIP-149
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> support
> >>>>>>>>>>>>>>>>>>>>>>>>> Rich
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> functions as well.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.  To as part of the Rich
> >>>>>> functions,
> >>>>>>>> we
> >>>>>>>>>>>>> provided
> >>>>>>>>>>>>>>> init
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (ProcessorContext)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method. Afterwards, Dammian
> >>>>>> suggested
> >>>>>>>>>> that
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext to users.
> >>> As a
> >>>>>>> result,
> >>>>>>>>>> we
> >>>>>>>>>>>>>> separated
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two separate KIPs, as it
> >>> seems
> >>>>> they
> >>>>>>> can
> >>>>>>>>>> be
> >>>>>>>>>>>>> solved
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> parallel.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - One approach we
> >> considered
> >>>> was
> >>>>> :
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
> >>>>>>> ValueMapperWithKey<K,
> >>>>>>>> V,
> >>>>>>>>>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 DSL users should not worry about -- and if
one really needs this, an advanced user can still insert a dummy
`transform` to request a commit from there.

-Matthias


On 10/18/17 5:39 AM, Jeyhun Karimov wrote:
> 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 argument to Rich
> methods:
> 
> instead of
> public interface RichValueMapper<V, VR, K> {
>     VR apply(final V value,
>              final K key,
>              final RecordContext recordContext);
> }
> 
> we can adopt
> 
> public interface RichValueMapper<V, VR, K> {
>     VR apply(final V value,
>              final K key,
>              final RecordContext recordContext,
>              final ProcessorContext processorContext);
> }
> 
> 
> However, in this case, a user can get confused as ProcessorContext and
> RecordContext share some methods with the same name.
> 
> 
> Cheers,
> Jeyhun
> 
> 
> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> 
> 
> On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com> wrote:
> 
>> 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 whole task up to this processed record, not only that single record
>> itself.
>>
>>
>> Guozhang
>>
>> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je...@gmail.com>
>> wrote:
>>
>>> 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<V1, V2, VR, K> {
>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>> RecordContext
>>>> recordContext);
>>>> }
>>>
>>>
>>>
>>> - Fixed.
>>>
>>>
>>> 2. Some of the listed functions are not necessary since their pairing
>> APIs
>>>> are being deprecated in 1.0 already:
>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
>> ?
>>>> super V, KR> selector,
>>>>                                    final Serde<KR> keySerde,
>>>>                                    final Serde<V> valSerde);
>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>                                  final RichValueJoiner<? super K, ?
>> super
>>>> V,
>>>> ? super VT, ? extends VR> joiner,
>>>>                                  final Serde<K> keySerde,
>>>>                                  final Serde<V> valSerde);
>>>
>>>
>>> -Fixed
>>>
>>> 3. For a few functions where we are adding three APIs for a combo of both
>>>> mapper / joiner, or both initializer / aggregator, or adder /
>> subtractor,
>>>> I'm wondering if we can just keep one that use "rich" functions for
>> both;
>>>> so that we can have less overloads and let users who only want to
>> access
>>>> one of them to just use dummy parameter declarations. For example:
>>>>
>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>> globalKTable,
>>>>                                  final RichKeyValueMapper<? super K, ?
>>>> super
>>>>  V, ? extends GK> keyValueMapper,
>>>>                                  final RichValueJoiner<? super K, ?
>> super
>>>> V,
>>>> ? super GV, ? extends RV> joiner);
>>>
>>>
>>>
>>> -Agreed. Fixed.
>>>
>>>
>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>> Initializer also "rich" functions? I.e.
>>>
>>>
>>> - It was a typo. Fixed.
>>>
>>>
>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
>>>> o.a.k.processor.
>>>>
>>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
>> to
>>>> RecordContext?
>>>>
>>>
>>> -
>>> Because it makes sense logically and  to reduce code maintenance (both
>>> interfaces have offset() timestamp() topic() partition() methods),  I
>>> inherit ProcessorContext from RecordContext.
>>> Since we need commit() method both in ProcessorContext and in
>> RecordContext
>>> I move commit() method to parent class (RecordContext).
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>>
>>>
>>> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>
>>>> 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<V1, V2, VR, K> {
>>>>     VR apply(final V1 value1, final V2 value2, final K key, final
>>>> RecordContext
>>>> recordContext);
>>>> }
>>>>
>>>> My motivation is that for lambda expression in J8, users that would not
>>>> care about the key but only the context, or vice versa, is likely to
>>> write
>>>> it as (value1, value2, dummy, context) -> ... than putting the dummy at
>>> the
>>>> beginning of the parameter list. Generally speaking we'd like to make
>> all
>>>> the "necessary" parameters prior to optional ones.
>>>>
>>>>
>>>> 2. Some of the listed functions are not necessary since their pairing
>>> APIs
>>>> are being deprecated in 1.0 already:
>>>>
>>>> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
>> ?
>>>> super V, KR> selector,
>>>>                                    final Serde<KR> keySerde,
>>>>                                    final Serde<V> valSerde);
>>>>
>>>> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>>>>                                  final RichValueJoiner<? super K, ?
>> super
>>>> V,
>>>> ? super VT, ? extends VR> joiner,
>>>>                                  final Serde<K> keySerde,
>>>>                                  final Serde<V> valSerde);
>>>>
>>>>
>>>>
>>>> 3. For a few functions where we are adding three APIs for a combo of
>> both
>>>> mapper / joiner, or both initializer / aggregator, or adder /
>> subtractor,
>>>> I'm wondering if we can just keep one that use "rich" functions for
>> both;
>>>> so that we can have less overloads and let users who only want to
>> access
>>>> one of them to just use dummy parameter declarations. For example:
>>>>
>>>>
>>>> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
>> globalKTable,
>>>>                                  final RichKeyValueMapper<? super K, ?
>>>> super
>>>>  V, ? extends GK> keyValueMapper,
>>>>                                  final RichValueJoiner<? super K, ?
>> super
>>>> V,
>>>> ? super GV, ? extends RV> joiner);
>>>>
>>>> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
>>>>                              final RichAggregator<? super K, ? super V,
>>> VR>
>>>> aggregator,
>>>>                              final Materialized<K, VR,
>>> KeyValueStore<Bytes,
>>>> byte[]>> materialized);
>>>>
>>>> Similarly for KGroupedTable, a bunch of aggregate() are deprecated so
>> we
>>> do
>>>> not need to add its rich functions any more.
>>>>
>>>>
>>>> 4. For TimeWindowedKStream, I'm wondering why we do not make its
>>>> Initializer also "rich" functions? I.e.
>>>>
>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
>>>> initializer,
>>>>                                        final RichAggregator<? super K,
>> ?
>>>> super V, VR> aggregator);
>>>> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
>>>> initializer,
>>>>                                        final RichAggregator<? super K,
>> ?
>>>> super V, VR> aggregator,
>>>>                                        final Materialized<K, VR,
>>>> WindowStore<Bytes, byte[]>> materialized);
>>>>
>>>>
>>>> 5. We need to move "RecordContext" from o.a.k.processor.internals to
>>>> o.a.k.processor.
>>>>
>>>> 6. I'm not clear why we want to move `commit()` from ProcessorContext
>> to
>>>> RecordContext? Conceptually I think it would better staying in the
>>>> ProcessorContext. Do you find this not doable in the internal
>>>> implementations?
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
>>>>
>>>>>    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 <
>> je.karimov@gmail.com
>>>>
>>>>> wrote:
>>>>>
>>>>>> 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 to KIP.
>>>>>>
>>>>>>
>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>                              final Aggregator<? super K, ? super
>> V,
>>>> VR>
>>>>>>> adder,
>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>
>>>>>>
>>>>>>
>>>>>> - Exactly. However, there are 2 Aggregator-type arguments in the
>>>> related
>>>>>> method. So, I had to overload all possible their Rich counterparts:
>>>>>>
>>>>>> // adder with non-rich, subtrctor is rich
>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>                              final Aggregator<? super K, ? super V,
>>> VR>
>>>>>> adder,
>>>>>>                              final RichAggregator<? super K, ?
>> super
>>> V,
>>>>> VR>
>>>>>> subtractor,
>>>>>>                              final Materialized<K, VR,
>>>>> KeyValueStore<Bytes,
>>>>>> byte[]>> materialized);
>>>>>>
>>>>>> // adder withrich, subtrctor is non-rich
>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>                              final RichAggregator<? super K, ?
>> super
>>> V,
>>>>> VR>
>>>>>> adder,
>>>>>>                              final Aggregator<? super K, ? super V,
>>> VR>
>>>>>> subtractor,
>>>>>>                              final Materialized<K, VR,
>>>>> KeyValueStore<Bytes,
>>>>>> byte[]>> materialized);
>>>>>>
>>>>>> // both adder and subtractor are rich
>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>                              final RichAggregator<? super K, ?
>> super
>>> V,
>>>>> VR>
>>>>>> adder,
>>>>>>                              final RichAggregator<? super K, ?
>> super
>>> V,
>>>>> VR>
>>>>>> subtractor,
>>>>>>                              final Materialized<K, VR,
>>>>> KeyValueStore<Bytes,
>>>>>> byte[]>> materialized);
>>>>>>
>>>>>>
>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>    void commit () {
>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>> supported
>>>>>> in
>>>>>>> this context");
>>>>>>> Is the exception going to be replaced with real code in the PR ?
>>>>>>
>>>>>>
>>>>>>
>>>>>> - I added some comments both inside and outside the code snippets
>> in
>>>> KIP.
>>>>>> Specifically, for the code snippet above, we add *commit()* method
>> to
>>>>>> *RecordContext* interface.
>>>>>> However, we want  *commit()* method to be used only for
>>> *RecordContext*
>>>>>> instances (at least for now), so we add
>> UnsupportedOperationException
>>>> in
>>>>>> all classes/interfaces that extend/implement *RecordContext.*
>>>>>> In general, 1) we make RecordContext publicly available within
>>>>>> ProcessorContext,  2) initialize its instance within all required
>>>>>> Processors and 3) pass it as an argument to the related Rich
>>> interfaces
>>>>>> inside Processors.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
>> wrote:
>>>>>>
>>>>>>> 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.
>>>>>>>
>>>>>>> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>>>>>>>                              final Aggregator<? super K, ? super
>> V,
>>>> VR>
>>>>>>> adder,
>>>>>>>
>>>>>>> The adder doesn't need to be RichAggregator ?
>>>>>>>
>>>>>>>   public RecordContext recordContext() {
>>>>>>>     return this.recordContext();
>>>>>>>
>>>>>>> Can you explain a bit about the above implementation ?
>>>>>>>
>>>>>>>    void commit () {
>>>>>>>      throw new UnsupportedOperationException("commit() is not
>>>>> supported
>>>>>> in
>>>>>>> this context");
>>>>>>>
>>>>>>> Is the exception going to be replaced with real code in the PR ?
>>>>>>>
>>>>>>> Cheers
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
>>>> je.karimov@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 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 <
>>>>> je.karimov@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> 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 <
>>> damian.guy@gmail.com
>>>>>
>>>>>>> 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 Jeyhun Karimov <
>>>> je.karimov@gmail.com
>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 merge this KIP into the upcoming release, as KIP-159 is
>>> not
>>>>>> voted
>>>>>>>> yet
>>>>>>>>>>> (because we want both KIP-149 and KIP-159 to be as an
>>> "atomic"
>>>>>>> merge).
>>>>>>>>>> So
>>>>>>>>>>> I decided to wait until KIP-182 gets stable (there are
>> some
>>>>> minor
>>>>>>>>>> updates
>>>>>>>>>>> AFAIK) and update the KIP accordingly. Please correct me
>> if
>>> I
>>>> am
>>>>>>> wrong
>>>>>>>>>> or I
>>>>>>>>>>> misunderstood.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
>>>>> damian.guy@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> +1
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
>>>>> wangguoz@gmail.com>
>>>>>>>>>> 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
>>> would
>>>>>> still
>>>>>>>>>> need to
>>>>>>>>>>>>> update your proposal for people to comment and vote
>> on.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
>>>>> yuzhihong@gmail.com
>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> +1
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One interface is cleaner.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
>>>>>>> bbejeck@gmail.com
>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> +1 for me on collapsing the RichXXXX and
>>>>> ValueXXXXWithKey
>>>>>>>>>>> interfaces
>>>>>>>>>>>>>> into 1
>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 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
>> 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
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> would not be a problem.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Our initial idea was similar to :
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
>>>>>>> implements
>>>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction {
>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So, we check the type of object, whether it is
>>>> RichXXX
>>>>>> or
>>>>>>>>>>>> XXXWithKey
>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>> the called method and continue accordingly.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If this is ok with the community, I would like
>> to
>>>>> revert
>>>>>>> the
>>>>>>>>>>>> current
>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>> to this again.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>>>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Jeyhun,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for sending out the update. I guess i
>> was
>>>>>>> thinking
>>>>>>>>>> more
>>>>>>>>>>>>> along
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> lines of option 2 where we collapse the
>> RichXXXX
>>>> and
>>>>>>>>>>>>> ValueXXXXWithKey
>>>>>>>>>>>>>>> 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?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 free to comment on this.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>> confluence/pages/viewpage.
>>>>>>>>>>>>>>>> action?pageId=73637757
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
>>> Karimov <
>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>> for one-shot solution for overloads
>>> issue.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, Jul 7, 2017 at 10:32 AM Damian
>>> Guy
>>>> <
>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>> overrides.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> About ProcessorContext vs
>>> RecordContext,
>>>>> you
>>>>>>> are
>>>>>>>>>>> right.
>>>>>>>>>>>> I
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>> implement a prototype to understand
>> the
>>>>> full
>>>>>>>>>> picture
>>>>>>>>>>> as
>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>> parts
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> KIP might not be as straightforward
>> as
>>> I
>>>>>>> thought.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jul 5, 2017 at 10:40 AM
>> Damian
>>>> Guy
>>>>> <
>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> Processor. Sot it doesn't make sense
>>> to
>>>>>> cast a
>>>>>>>>>>>>>>> ProcessorContext
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> RecordContext.
>>>>>>>>>>>>>>>>>>>>>>>> You mentioned above passing the
>>>>>>>>>>>> InternalProcessorContext
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> init()
>>>>>>>>>>>>>>>>>>>>>>>> calls. It is internal for a reason
>>> and i
>>>>>> think
>>>>>>>> it
>>>>>>>>>>>> should
>>>>>>>>>>>>>>> remain
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>> It might be better to move the
>>>>>> recordContext()
>>>>>>>>>> method
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>> InternalProcessorContext to
>>>>>> ProcessorContext.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> In the KIP you have an example
>>> showing:
>>>>>>>>>>>>>>>>>>>>>>>> richMapper.init((RecordContext)
>>>>>>>> processorContext);
>>>>>>>>>>>>>>>>>>>>>>>> But the interface is:
>>>>>>>>>>>>>>>>>>>>>>>> public interface RichValueMapper<V,
>>> VR>
>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final V value, final
>>>>>>> RecordContext
>>>>>>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>> i.e., there is no init(...), besides
>>> as
>>>>>> above
>>>>>>>> this
>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>> sense.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
>>>>> Karimov <
>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>> Can't
>>>>>>>>>>> we
>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>> parameter to init() method of
>>>> processors?
>>>>>>> Then
>>>>>>>> we
>>>>>>>>>>>> would
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> able
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>> RecordContext easily with just a
>>> method
>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Jun 29, 2017 at 10:14 PM
>>>> Matthias
>>>>>> J.
>>>>>>>> Sax
>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>> be provided by the consecutive
>> call
>>> to
>>>>>>>> aggregate
>>>>>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/29/17 1:11 PM, Matthias J.
>> Sax
>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>> implement
>>>>>>>>>> both.
>>>>>>>>>>>> This
>>>>>>>>>>>>>>> allows
>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> cast
>>>>>>>>>>>>>>>>>>>>>>>>>>> it to `RecordContext` and thus
>>> limit
>>>>> the
>>>>>>>>>> visible
>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/27/17 1:35 PM, Jeyhun
>> Karimov
>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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 address this issue
>>> as
>>>>> well
>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue as it is?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
>>>>>> Guozhang
>>>>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> LGTM. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 2:20
>> PM,
>>>>> Jeyhun
>>>>>>>>>> Karimov
>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>> KAFKA-3907
>>>>>> [1]
>>>>>>>> can
>>>>>>>>>>> also
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> integrated
>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP by adding related method
>>>> inside
>>>>>>>>>>> RecordContext
>>>>>>>>>>>>>>>>> interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
>> PM
>>>>>> Matthias
>>>>>>>> J.
>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>> information
>>>>>>>>>>> only.
>>>>>>>>>>>>> If
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stateful computation you
>> should
>>>>>> rather
>>>>>>>> use
>>>>>>>>>>>>>>> #transform().
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, as pointed out,
>> we
>>>>> would
>>>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>>> switch
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> supplier-pattern introducing
>>> many
>>>>>> more
>>>>>>>>>>>> overloads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For those reason, I advocate
>>> for
>>>> a
>>>>>>> simple
>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>> with a
>>>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that passes in a
>> RecordContext
>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
>>> Wang
>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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, so I
>>> will
>>>>> not
>>>>>>>>>> mention
>>>>>>>>>>>>> them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The related discussions are
>>>>> spread
>>>>>> in
>>>>>>>>>>> KIP-149
>>>>>>>>>>>>> and
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KIP-159)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion threads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. The idea of rich
>> functions
>>>>> came
>>>>>>> into
>>>>>>>>>> the
>>>>>>>>>>>>> stage
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> KIP-149,
>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion thread. As a
>>> result
>>>> we
>>>>>>>>>> extended
>>>>>>>>>>>>> KIP-149
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>>>>>>>>>>>> Rich
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> functions as well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.  To as part of the Rich
>>>>>> functions,
>>>>>>>> we
>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>> init
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (ProcessorContext)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method. Afterwards, Dammian
>>>>>> suggested
>>>>>>>>>> that
>>>>>>>>>>> we
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext to users.
>>> As a
>>>>>>> result,
>>>>>>>>>> we
>>>>>>>>>>>>>> separated
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two separate KIPs, as it
>>> seems
>>>>> they
>>>>>>> can
>>>>>>>>>> be
>>>>>>>>>>>>> solved
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> parallel.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - One approach we
>> considered
>>>> was
>>>>> :
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
>>>>>>> ValueMapperWithKey<K,
>>>>>>>> V,
>>>>>>>>>>> VR>
>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     VR apply(final K key,
>>>> final V
>>>>>>>> value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
>>>>> RichValueMapper<K,
>>>>>>> V,
>>>>>>>>>> VR>
>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>> RichFunction{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
>>> RichFunction {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     void init(RecordContext
>>>>>>>>>> recordContext);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     void close();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> public interface
>>> RecordContext
>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     String applicationId();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     TaskId taskId();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     StreamsMetrics
>> metrics();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     String topic();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     int partition();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     long offset();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     long timestamp();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     Map<String, Object>
>>>>>> appConfigs();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>     Map<String, Object>
>>>>>>>>>>>>>> appConfigsWithPrefix(String
>>>>>>>>>>>>>>>>>>>>>> prefix);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>
> 


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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 argument to Rich
methods:

instead of
public interface RichValueMapper<V, VR, K> {
    VR apply(final V value,
             final K key,
             final RecordContext recordContext);
}

we can adopt

public interface RichValueMapper<V, VR, K> {
    VR apply(final V value,
             final K key,
             final RecordContext recordContext,
             final ProcessorContext processorContext);
}


However, in this case, a user can get confused as ProcessorContext and
RecordContext share some methods with the same name.


Cheers,
Jeyhun


[1] https://issues.apache.org/jira/browse/KAFKA-3907


On Tue, Oct 17, 2017 at 3:19 AM Guozhang Wang <wa...@gmail.com> wrote:

> 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 whole task up to this processed record, not only that single record
> itself.
>
>
> Guozhang
>
> On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je...@gmail.com>
> wrote:
>
> > 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<V1, V2, VR, K> {
> > >     VR apply(final V1 value1, final V2 value2, final K key, final
> > > RecordContext
> > > recordContext);
> > > }
> >
> >
> >
> > - Fixed.
> >
> >
> > 2. Some of the listed functions are not necessary since their pairing
> APIs
> > > are being deprecated in 1.0 already:
> > > <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
> ?
> > > super V, KR> selector,
> > >                                    final Serde<KR> keySerde,
> > >                                    final Serde<V> valSerde);
> > > <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >                                  final RichValueJoiner<? super K, ?
> super
> > > V,
> > > ? super VT, ? extends VR> joiner,
> > >                                  final Serde<K> keySerde,
> > >                                  final Serde<V> valSerde);
> >
> >
> > -Fixed
> >
> > 3. For a few functions where we are adding three APIs for a combo of both
> > > mapper / joiner, or both initializer / aggregator, or adder /
> subtractor,
> > > I'm wondering if we can just keep one that use "rich" functions for
> both;
> > > so that we can have less overloads and let users who only want to
> access
> > > one of them to just use dummy parameter declarations. For example:
> > >
> > > <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> globalKTable,
> > >                                  final RichKeyValueMapper<? super K, ?
> > > super
> > >  V, ? extends GK> keyValueMapper,
> > >                                  final RichValueJoiner<? super K, ?
> super
> > > V,
> > > ? super GV, ? extends RV> joiner);
> >
> >
> >
> > -Agreed. Fixed.
> >
> >
> > 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > > Initializer also "rich" functions? I.e.
> >
> >
> > - It was a typo. Fixed.
> >
> >
> > 5. We need to move "RecordContext" from o.a.k.processor.internals to
> > > o.a.k.processor.
> > >
> > > 6. I'm not clear why we want to move `commit()` from ProcessorContext
> to
> > > RecordContext?
> > >
> >
> > -
> > Because it makes sense logically and  to reduce code maintenance (both
> > interfaces have offset() timestamp() topic() partition() methods),  I
> > inherit ProcessorContext from RecordContext.
> > Since we need commit() method both in ProcessorContext and in
> RecordContext
> > I move commit() method to parent class (RecordContext).
> >
> >
> > Cheers,
> > Jeyhun
> >
> >
> >
> > On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> >
> > > 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<V1, V2, VR, K> {
> > >     VR apply(final V1 value1, final V2 value2, final K key, final
> > > RecordContext
> > > recordContext);
> > > }
> > >
> > > My motivation is that for lambda expression in J8, users that would not
> > > care about the key but only the context, or vice versa, is likely to
> > write
> > > it as (value1, value2, dummy, context) -> ... than putting the dummy at
> > the
> > > beginning of the parameter list. Generally speaking we'd like to make
> all
> > > the "necessary" parameters prior to optional ones.
> > >
> > >
> > > 2. Some of the listed functions are not necessary since their pairing
> > APIs
> > > are being deprecated in 1.0 already:
> > >
> > > <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K,
> ?
> > > super V, KR> selector,
> > >                                    final Serde<KR> keySerde,
> > >                                    final Serde<V> valSerde);
> > >
> > > <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> > >                                  final RichValueJoiner<? super K, ?
> super
> > > V,
> > > ? super VT, ? extends VR> joiner,
> > >                                  final Serde<K> keySerde,
> > >                                  final Serde<V> valSerde);
> > >
> > >
> > >
> > > 3. For a few functions where we are adding three APIs for a combo of
> both
> > > mapper / joiner, or both initializer / aggregator, or adder /
> subtractor,
> > > I'm wondering if we can just keep one that use "rich" functions for
> both;
> > > so that we can have less overloads and let users who only want to
> access
> > > one of them to just use dummy parameter declarations. For example:
> > >
> > >
> > > <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV>
> globalKTable,
> > >                                  final RichKeyValueMapper<? super K, ?
> > > super
> > >  V, ? extends GK> keyValueMapper,
> > >                                  final RichValueJoiner<? super K, ?
> super
> > > V,
> > > ? super GV, ? extends RV> joiner);
> > >
> > > <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
> > >                              final RichAggregator<? super K, ? super V,
> > VR>
> > > aggregator,
> > >                              final Materialized<K, VR,
> > KeyValueStore<Bytes,
> > > byte[]>> materialized);
> > >
> > > Similarly for KGroupedTable, a bunch of aggregate() are deprecated so
> we
> > do
> > > not need to add its rich functions any more.
> > >
> > >
> > > 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > > Initializer also "rich" functions? I.e.
> > >
> > > <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> > > initializer,
> > >                                        final RichAggregator<? super K,
> ?
> > > super V, VR> aggregator);
> > > <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> > > initializer,
> > >                                        final RichAggregator<? super K,
> ?
> > > super V, VR> aggregator,
> > >                                        final Materialized<K, VR,
> > > WindowStore<Bytes, byte[]>> materialized);
> > >
> > >
> > > 5. We need to move "RecordContext" from o.a.k.processor.internals to
> > > o.a.k.processor.
> > >
> > > 6. I'm not clear why we want to move `commit()` from ProcessorContext
> to
> > > RecordContext? Conceptually I think it would better staying in the
> > > ProcessorContext. Do you find this not doable in the internal
> > > implementations?
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
> > >
> > > >    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 <
> je.karimov@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > 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 to KIP.
> > > > >
> > > > >
> > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > > >                              final Aggregator<? super K, ? super
> V,
> > > VR>
> > > > > > adder,
> > > > > > The adder doesn't need to be RichAggregator ?
> > > > >
> > > > >
> > > > >
> > > > > - Exactly. However, there are 2 Aggregator-type arguments in the
> > > related
> > > > > method. So, I had to overload all possible their Rich counterparts:
> > > > >
> > > > > // adder with non-rich, subtrctor is rich
> > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > >                              final Aggregator<? super K, ? super V,
> > VR>
> > > > > adder,
> > > > >                              final RichAggregator<? super K, ?
> super
> > V,
> > > > VR>
> > > > > subtractor,
> > > > >                              final Materialized<K, VR,
> > > > KeyValueStore<Bytes,
> > > > > byte[]>> materialized);
> > > > >
> > > > > // adder withrich, subtrctor is non-rich
> > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > >                              final RichAggregator<? super K, ?
> super
> > V,
> > > > VR>
> > > > > adder,
> > > > >                              final Aggregator<? super K, ? super V,
> > VR>
> > > > > subtractor,
> > > > >                              final Materialized<K, VR,
> > > > KeyValueStore<Bytes,
> > > > > byte[]>> materialized);
> > > > >
> > > > > // both adder and subtractor are rich
> > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > >                              final RichAggregator<? super K, ?
> super
> > V,
> > > > VR>
> > > > > adder,
> > > > >                              final RichAggregator<? super K, ?
> super
> > V,
> > > > VR>
> > > > > subtractor,
> > > > >                              final Materialized<K, VR,
> > > > KeyValueStore<Bytes,
> > > > > byte[]>> materialized);
> > > > >
> > > > >
> > > > > Can you explain a bit about the above implementation ?
> > > > > >    void commit () {
> > > > > >      throw new UnsupportedOperationException("commit() is not
> > > > supported
> > > > > in
> > > > > > this context");
> > > > > > Is the exception going to be replaced with real code in the PR ?
> > > > >
> > > > >
> > > > >
> > > > > - I added some comments both inside and outside the code snippets
> in
> > > KIP.
> > > > > Specifically, for the code snippet above, we add *commit()* method
> to
> > > > > *RecordContext* interface.
> > > > > However, we want  *commit()* method to be used only for
> > *RecordContext*
> > > > > instances (at least for now), so we add
> UnsupportedOperationException
> > > in
> > > > > all classes/interfaces that extend/implement *RecordContext.*
> > > > > In general, 1) we make RecordContext publicly available within
> > > > > ProcessorContext,  2) initialize its instance within all required
> > > > > Processors and 3) pass it as an argument to the related Rich
> > interfaces
> > > > > inside Processors.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > Cheers,
> > > > > Jeyhun
> > > > >
> > > > > On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com>
> wrote:
> > > > >
> > > > > > 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.
> > > > > >
> > > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > > >                              final Aggregator<? super K, ? super
> V,
> > > VR>
> > > > > > adder,
> > > > > >
> > > > > > The adder doesn't need to be RichAggregator ?
> > > > > >
> > > > > >   public RecordContext recordContext() {
> > > > > >     return this.recordContext();
> > > > > >
> > > > > > Can you explain a bit about the above implementation ?
> > > > > >
> > > > > >    void commit () {
> > > > > >      throw new UnsupportedOperationException("commit() is not
> > > > supported
> > > > > in
> > > > > > this context");
> > > > > >
> > > > > > Is the exception going to be replaced with real code in the PR ?
> > > > > >
> > > > > > Cheers
> > > > > >
> > > > > >
> > > > > > On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> > > je.karimov@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > 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 <
> > > > je.karimov@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > 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 <
> > damian.guy@gmail.com
> > > >
> > > > > > 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 Jeyhun Karimov <
> > > je.karimov@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >>
> > > > > > > >> > 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 merge this KIP into the upcoming release, as KIP-159 is
> > not
> > > > > voted
> > > > > > > yet
> > > > > > > >> > (because we want both KIP-149 and KIP-159 to be as an
> > "atomic"
> > > > > > merge).
> > > > > > > >> So
> > > > > > > >> > I decided to wait until KIP-182 gets stable (there are
> some
> > > > minor
> > > > > > > >> updates
> > > > > > > >> > AFAIK) and update the KIP accordingly. Please correct me
> if
> > I
> > > am
> > > > > > wrong
> > > > > > > >> or I
> > > > > > > >> > misunderstood.
> > > > > > > >> >
> > > > > > > >> > Cheers,
> > > > > > > >> > Jeyhun
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > > > damian.guy@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >
> > > > > > > >> > > +1
> > > > > > > >> > >
> > > > > > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > > >> 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
> > would
> > > > > still
> > > > > > > >> need to
> > > > > > > >> > > > update your proposal for people to comment and vote
> on.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > Guozhang
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > > > yuzhihong@gmail.com
> > > > > >
> > > > > > > >> wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > +1
> > > > > > > >> > > > >
> > > > > > > >> > > > > One interface is cleaner.
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > > > > > bbejeck@gmail.com
> > > > > > > >
> > > > > > > >> > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > +1 for me on collapsing the RichXXXX and
> > > > ValueXXXXWithKey
> > > > > > > >> > interfaces
> > > > > > > >> > > > > into 1
> > > > > > > >> > > > > > interface.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks,
> > > > > > > >> > > > > > Bill
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > > > > > >> > > je.karimov@gmail.com
> > > > > > > >> > > > >
> > > > > > > >> > > > > > 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
> 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
> > > > > > > >> > > > > > this
> > > > > > > >> > > > > > > would not be a problem.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Our initial idea was similar to :
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> > > > > > implements
> > > > > > > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > > > >> > > > > > > ......
> > > > > > > >> > > > > > > }
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > So, we check the type of object, whether it is
> > > RichXXX
> > > > > or
> > > > > > > >> > > XXXWithKey
> > > > > > > >> > > > > > inside
> > > > > > > >> > > > > > > the called method and continue accordingly.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > If this is ok with the community, I would like
> to
> > > > revert
> > > > > > the
> > > > > > > >> > > current
> > > > > > > >> > > > > > design
> > > > > > > >> > > > > > > to this again.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Cheers,
> > > > > > > >> > > > > > > Jeyhun
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > > > > > >> damian.guy@gmail.com
> > > > > > > >> > >
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > Hi Jeyhun,
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Thanks for sending out the update. I guess i
> was
> > > > > > thinking
> > > > > > > >> more
> > > > > > > >> > > > along
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > lines of option 2 where we collapse the
> RichXXXX
> > > and
> > > > > > > >> > > > ValueXXXXWithKey
> > > > > > > >> > > > > > 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?
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Thanks,
> > > > > > > >> > > > > > > > Damian
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > > > > > >> > > je.karimov@gmail.com>
> > > > > > > >> > > > > > > wrote:
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > > 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 free to comment on this.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > [1]
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > https://cwiki.apache.org/
> > > confluence/pages/viewpage.
> > > > > > > >> > > > > > > action?pageId=73637757
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > Cheers,
> > > > > > > >> > > > > > > > > Jeyhun
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> > Karimov <
> > > > > > > >> > > > > je.karimov@gmail.com
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > > wrote:
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > > 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 <
> > > > > > > >> > > > > > > matthias@confluent.io>
> > > > > > > >> > > > > > > > > > wrote:
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > > >> 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
> > > > > > > >> > > > > > > > > >> > for one-shot solution for overloads
> > issue.
> > > > > > > >> > > > > > > > > >> >
> > > > > > > >> > > > > > > > > >> > Cheers,
> > > > > > > >> > > > > > > > > >> > Jeyhun
> > > > > > > >> > > > > > > > > >> >
> > > > > > > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian
> > Guy
> > > <
> > > > > > > >> > > > > > damian.guy@gmail.com>
> > > > > > > >> > > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > > >> >
> > > > > > > >> > > > > > > > > >> >> 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
> > > > > > > >> > > > > > > > > overrides.
> > > > > > > >> > > > > > > > > >> >>
> > > > > > > >> > > > > > > > > >> >> Thanks,
> > > > > > > >> > > > > > > > > >> >> Damian
> > > > > > > >> > > > > > > > > >> >>
> > > > > > > >> > > > > > > > > >> >>
> > > > > > > >> > > > > > > > > >> >>> About ProcessorContext vs
> > RecordContext,
> > > > you
> > > > > > are
> > > > > > > >> > right.
> > > > > > > >> > > I
> > > > > > > >> > > > > > think
> > > > > > > >> > > > > > > I
> > > > > > > >> > > > > > > > > >> need to
> > > > > > > >> > > > > > > > > >> >>> implement a prototype to understand
> the
> > > > full
> > > > > > > >> picture
> > > > > > > >> > as
> > > > > > > >> > > > some
> > > > > > > >> > > > > > > parts
> > > > > > > >> > > > > > > > > of
> > > > > > > >> > > > > > > > > >> the
> > > > > > > >> > > > > > > > > >> >>> KIP might not be as straightforward
> as
> > I
> > > > > > thought.
> > > > > > > >> > > > > > > > > >> >>>
> > > > > > > >> > > > > > > > > >> >>>
> > > > > > > >> > > > > > > > > >> >>> Cheers,
> > > > > > > >> > > > > > > > > >> >>> Jeyhun
> > > > > > > >> > > > > > > > > >> >>>
> > > > > > > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM
> Damian
> > > Guy
> > > > <
> > > > > > > >> > > > > > > damian.guy@gmail.com>
> > > > > > > >> > > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > > >> >>>
> > > > > > > >> > > > > > > > > >> >>>> 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
> > > > > > > >> > > > > lifetime
> > > > > > > >> > > > > > of
> > > > > > > >> > > > > > > > the
> > > > > > > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense
> > to
> > > > > cast a
> > > > > > > >> > > > > > ProcessorContext
> > > > > > > >> > > > > > > > to
> > > > > > > >> > > > > > > > > a
> > > > > > > >> > > > > > > > > >> >>>> RecordContext.
> > > > > > > >> > > > > > > > > >> >>>> You mentioned above passing the
> > > > > > > >> > > InternalProcessorContext
> > > > > > > >> > > > to
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > > >> init()
> > > > > > > >> > > > > > > > > >> >>>> calls. It is internal for a reason
> > and i
> > > > > think
> > > > > > > it
> > > > > > > >> > > should
> > > > > > > >> > > > > > remain
> > > > > > > >> > > > > > > > > that
> > > > > > > >> > > > > > > > > >> >> way.
> > > > > > > >> > > > > > > > > >> >>>> It might be better to move the
> > > > > recordContext()
> > > > > > > >> method
> > > > > > > >> > > > from
> > > > > > > >> > > > > > > > > >> >>>> InternalProcessorContext to
> > > > > ProcessorContext.
> > > > > > > >> > > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > > >> >>>> In the KIP you have an example
> > showing:
> > > > > > > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > > > > > > processorContext);
> > > > > > > >> > > > > > > > > >> >>>> But the interface is:
> > > > > > > >> > > > > > > > > >> >>>> public interface RichValueMapper<V,
> > VR>
> > > {
> > > > > > > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> > > > > > RecordContext
> > > > > > > >> > > > > > recordContext);
> > > > > > > >> > > > > > > > > >> >>>> }
> > > > > > > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides
> > as
> > > > > above
> > > > > > > this
> > > > > > > >> > > > wouldn't
> > > > > > > >> > > > > > > make
> > > > > > > >> > > > > > > > > >> sense.
> > > > > > > >> > > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > > >> >>>> Thanks,
> > > > > > > >> > > > > > > > > >> >>>> Damian
> > > > > > > >> > > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> > > > Karimov <
> > > > > > > >> > > > > > > je.karimov@gmail.com
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > >> >> wrote:
> > > > > > > >> > > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > > >> >>>>> 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.
> > > > > > > >> Can't
> > > > > > > >> > we
> > > > > > > >> > > > pass
> > > > > > > >> > > > > > > this
> > > > > > > >> > > > > > > > > as a
> > > > > > > >> > > > > > > > > >> >>>>> parameter to init() method of
> > > processors?
> > > > > > Then
> > > > > > > we
> > > > > > > >> > > would
> > > > > > > >> > > > be
> > > > > > > >> > > > > > > able
> > > > > > > >> > > > > > > > to
> > > > > > > >> > > > > > > > > >> >> get
> > > > > > > >> > > > > > > > > >> >>>>> RecordContext easily with just a
> > method
> > > > > call.
> > > > > > > >> > > > > > > > > >> >>>>>
> > > > > > > >> > > > > > > > > >> >>>>>
> > > > > > > >> > > > > > > > > >> >>>>> Cheers,
> > > > > > > >> > > > > > > > > >> >>>>> Jeyhun
> > > > > > > >> > > > > > > > > >> >>>>>
> > > > > > > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM
> > > Matthias
> > > > > J.
> > > > > > > Sax
> > > > > > > >> <
> > > > > > > >> > > > > > > > > >> >>> matthias@confluent.io>
> > > > > > > >> > > > > > > > > >> >>>>> wrote:
> > > > > > > >> > > > > > > > > >> >>>>>
> > > > > > > >> > > > > > > > > >> >>>>>> 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
> > > > > > > >> > > > > > > > > >> >>> will
> > > > > > > >> > > > > > > > > >> >>>>>> be provided by the consecutive
> call
> > to
> > > > > > > aggregate
> > > > > > > >> > > > anyways.
> > > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>> -Matthias
> > > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J.
> Sax
> > > > wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>> 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
> > > > > implement
> > > > > > > >> both.
> > > > > > > >> > > This
> > > > > > > >> > > > > > allows
> > > > > > > >> > > > > > > > us
> > > > > > > >> > > > > > > > > >> >> to
> > > > > > > >> > > > > > > > > >> >>>> cast
> > > > > > > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus
> > limit
> > > > the
> > > > > > > >> visible
> > > > > > > >> > > > scope.
> > > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>> -Matthias
> > > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun
> Karimov
> > > > wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>>> 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 address this issue
> > as
> > > > well
> > > > > > or
> > > > > > > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>> Cheers,
> > > > > > > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> > > > > Guozhang
> > > > > > > >> Wang <
> > > > > > > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > > > > > >> > > > > > > > > >> >>>>>> wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20
> PM,
> > > > Jeyhun
> > > > > > > >> Karimov
> > > > > > > >> > <
> > > > > > > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > > >> > > > > > > > > >> >>>>>>>>> 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
> > > KAFKA-3907
> > > > > [1]
> > > > > > > can
> > > > > > > >> > also
> > > > > > > >> > > > be
> > > > > > > >> > > > > > > > > >> >> integrated
> > > > > > > >> > > > > > > > > >> >>>> into
> > > > > > > >> > > > > > > > > >> >>>>>> the
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method
> > > inside
> > > > > > > >> > RecordContext
> > > > > > > >> > > > > > > > interface.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> [1]
> > > > > > > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50
> PM
> > > > > Matthias
> > > > > > > J.
> > > > > > > >> > Sax <
> > > > > > > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> 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
> > > > > > > information
> > > > > > > >> > only.
> > > > > > > >> > > > If
> > > > > > > >> > > > > > you
> > > > > > > >> > > > > > > > want
> > > > > > > >> > > > > > > > > >> >> to
> > > > > > > >> > > > > > > > > >> >>>> do
> > > > > > > >> > > > > > > > > >> >>>>> a
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you
> should
> > > > > rather
> > > > > > > use
> > > > > > > >> > > > > > #transform().
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out,
> we
> > > > would
> > > > > > > need
> > > > > > > >> to
> > > > > > > >> > > > switch
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > a
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing
> > many
> > > > > more
> > > > > > > >> > > overloads.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate
> > for
> > > a
> > > > > > simple
> > > > > > > >> > > > interface
> > > > > > > >> > > > > > > with a
> > > > > > > >> > > > > > > > > >> >>> single
> > > > > > > >> > > > > > > > > >> >>>>>>>>> method
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> that passes in a
> RecordContext
> > > > > object.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
> > Wang
> > > > > wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > > > > > > >> > > <
> > > > > > > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I
> > will
> > > > not
> > > > > > > >> mention
> > > > > > > >> > > > them.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are
> > > > spread
> > > > > in
> > > > > > > >> > KIP-149
> > > > > > > >> > > > and
> > > > > > > >> > > > > in
> > > > > > > >> > > > > > > > this
> > > > > > > >> > > > > > > > > >> >> KIP
> > > > > > > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich
> functions
> > > > came
> > > > > > into
> > > > > > > >> the
> > > > > > > >> > > > stage
> > > > > > > >> > > > > > with
> > > > > > > >> > > > > > > > > >> >>> KIP-149,
> > > > > > > >> > > > > > > > > >> >>>>> in
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a
> > result
> > > we
> > > > > > > >> extended
> > > > > > > >> > > > KIP-149
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > > >> >> support
> > > > > > > >> > > > > > > > > >> >>>>> Rich
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich
> > > > > functions,
> > > > > > > we
> > > > > > > >> > > > provided
> > > > > > > >> > > > > > init
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian
> > > > > suggested
> > > > > > > >> that
> > > > > > > >> > we
> > > > > > > >> > > > > should
> > > > > > > >> > > > > > > not
> > > > > > > >> > > > > > > > > >> >>>> provide
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users.
> > As a
> > > > > > result,
> > > > > > > >> we
> > > > > > > >> > > > > separated
> > > > > > > >> > > > > > > the
> > > > > > > >> > > > > > > > > >> >> two
> > > > > > > >> > > > > > > > > >> >>>>>>>>> problems
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>> into
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it
> > seems
> > > > they
> > > > > > can
> > > > > > > >> be
> > > > > > > >> > > > solved
> > > > > > > >> > > > > in
> > > > > > > >> > > > > > > > > >> >>> parallel.
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we
> considered
> > > was
> > > > :
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > > > > ValueMapperWithKey<K,
> > > > > > > V,
> > > > > > > >> > VR>
> > > > > > > >> > > {
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key,
> > > final V
> > > > > > > value);
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > > RichValueMapper<K,
> > > > > > V,
> > > > > > > >> VR>
> > > > > > > >> > > > extends
> > > > > > > >> > > > > > > > > >> >>>> RichFunction{
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > RichFunction {
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > > > > > > >> recordContext);
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > RecordContext
> > > {
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics
> metrics();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > > appConfigs();
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > > > > >> > > > > appConfigsWithPrefix(String
> > > > > > > >> > > > > > > > > >> >> prefix);
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> >

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

Posted by Guozhang Wang <wa...@gmail.com>.
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 whole task up to this processed record, not only that single record
itself.


Guozhang

On Mon, Oct 16, 2017 at 9:19 AM, Jeyhun Karimov <je...@gmail.com>
wrote:

> 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<V1, V2, VR, K> {
> >     VR apply(final V1 value1, final V2 value2, final K key, final
> > RecordContext
> > recordContext);
> > }
>
>
>
> - Fixed.
>
>
> 2. Some of the listed functions are not necessary since their pairing APIs
> > are being deprecated in 1.0 already:
> > <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ?
> > super V, KR> selector,
> >                                    final Serde<KR> keySerde,
> >                                    final Serde<V> valSerde);
> > <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >                                  final RichValueJoiner<? super K, ? super
> > V,
> > ? super VT, ? extends VR> joiner,
> >                                  final Serde<K> keySerde,
> >                                  final Serde<V> valSerde);
>
>
> -Fixed
>
> 3. For a few functions where we are adding three APIs for a combo of both
> > mapper / joiner, or both initializer / aggregator, or adder / subtractor,
> > I'm wondering if we can just keep one that use "rich" functions for both;
> > so that we can have less overloads and let users who only want to access
> > one of them to just use dummy parameter declarations. For example:
> >
> > <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
> >                                  final RichKeyValueMapper<? super K, ?
> > super
> >  V, ? extends GK> keyValueMapper,
> >                                  final RichValueJoiner<? super K, ? super
> > V,
> > ? super GV, ? extends RV> joiner);
>
>
>
> -Agreed. Fixed.
>
>
> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > Initializer also "rich" functions? I.e.
>
>
> - It was a typo. Fixed.
>
>
> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> > o.a.k.processor.
> >
> > 6. I'm not clear why we want to move `commit()` from ProcessorContext to
> > RecordContext?
> >
>
> -
> Because it makes sense logically and  to reduce code maintenance (both
> interfaces have offset() timestamp() topic() partition() methods),  I
> inherit ProcessorContext from RecordContext.
> Since we need commit() method both in ProcessorContext and in RecordContext
> I move commit() method to parent class (RecordContext).
>
>
> Cheers,
> Jeyhun
>
>
>
> On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > 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<V1, V2, VR, K> {
> >     VR apply(final V1 value1, final V2 value2, final K key, final
> > RecordContext
> > recordContext);
> > }
> >
> > My motivation is that for lambda expression in J8, users that would not
> > care about the key but only the context, or vice versa, is likely to
> write
> > it as (value1, value2, dummy, context) -> ... than putting the dummy at
> the
> > beginning of the parameter list. Generally speaking we'd like to make all
> > the "necessary" parameters prior to optional ones.
> >
> >
> > 2. Some of the listed functions are not necessary since their pairing
> APIs
> > are being deprecated in 1.0 already:
> >
> > <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ?
> > super V, KR> selector,
> >                                    final Serde<KR> keySerde,
> >                                    final Serde<V> valSerde);
> >
> > <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
> >                                  final RichValueJoiner<? super K, ? super
> > V,
> > ? super VT, ? extends VR> joiner,
> >                                  final Serde<K> keySerde,
> >                                  final Serde<V> valSerde);
> >
> >
> >
> > 3. For a few functions where we are adding three APIs for a combo of both
> > mapper / joiner, or both initializer / aggregator, or adder / subtractor,
> > I'm wondering if we can just keep one that use "rich" functions for both;
> > so that we can have less overloads and let users who only want to access
> > one of them to just use dummy parameter declarations. For example:
> >
> >
> > <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
> >                                  final RichKeyValueMapper<? super K, ?
> > super
> >  V, ? extends GK> keyValueMapper,
> >                                  final RichValueJoiner<? super K, ? super
> > V,
> > ? super GV, ? extends RV> joiner);
> >
> > <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
> >                              final RichAggregator<? super K, ? super V,
> VR>
> > aggregator,
> >                              final Materialized<K, VR,
> KeyValueStore<Bytes,
> > byte[]>> materialized);
> >
> > Similarly for KGroupedTable, a bunch of aggregate() are deprecated so we
> do
> > not need to add its rich functions any more.
> >
> >
> > 4. For TimeWindowedKStream, I'm wondering why we do not make its
> > Initializer also "rich" functions? I.e.
> >
> > <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> > initializer,
> >                                        final RichAggregator<? super K, ?
> > super V, VR> aggregator);
> > <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> > initializer,
> >                                        final RichAggregator<? super K, ?
> > super V, VR> aggregator,
> >                                        final Materialized<K, VR,
> > WindowStore<Bytes, byte[]>> materialized);
> >
> >
> > 5. We need to move "RecordContext" from o.a.k.processor.internals to
> > o.a.k.processor.
> >
> > 6. I'm not clear why we want to move `commit()` from ProcessorContext to
> > RecordContext? Conceptually I think it would better staying in the
> > ProcessorContext. Do you find this not doable in the internal
> > implementations?
> >
> >
> > Guozhang
> >
> >
> >
> > On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> > >    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 <je.karimov@gmail.com
> >
> > > wrote:
> > >
> > > > 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 to KIP.
> > > >
> > > >
> > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > >                              final Aggregator<? super K, ? super V,
> > VR>
> > > > > adder,
> > > > > The adder doesn't need to be RichAggregator ?
> > > >
> > > >
> > > >
> > > > - Exactly. However, there are 2 Aggregator-type arguments in the
> > related
> > > > method. So, I had to overload all possible their Rich counterparts:
> > > >
> > > > // adder with non-rich, subtrctor is rich
> > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > >                              final Aggregator<? super K, ? super V,
> VR>
> > > > adder,
> > > >                              final RichAggregator<? super K, ? super
> V,
> > > VR>
> > > > subtractor,
> > > >                              final Materialized<K, VR,
> > > KeyValueStore<Bytes,
> > > > byte[]>> materialized);
> > > >
> > > > // adder withrich, subtrctor is non-rich
> > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > >                              final RichAggregator<? super K, ? super
> V,
> > > VR>
> > > > adder,
> > > >                              final Aggregator<? super K, ? super V,
> VR>
> > > > subtractor,
> > > >                              final Materialized<K, VR,
> > > KeyValueStore<Bytes,
> > > > byte[]>> materialized);
> > > >
> > > > // both adder and subtractor are rich
> > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > >                              final RichAggregator<? super K, ? super
> V,
> > > VR>
> > > > adder,
> > > >                              final RichAggregator<? super K, ? super
> V,
> > > VR>
> > > > subtractor,
> > > >                              final Materialized<K, VR,
> > > KeyValueStore<Bytes,
> > > > byte[]>> materialized);
> > > >
> > > >
> > > > Can you explain a bit about the above implementation ?
> > > > >    void commit () {
> > > > >      throw new UnsupportedOperationException("commit() is not
> > > supported
> > > > in
> > > > > this context");
> > > > > Is the exception going to be replaced with real code in the PR ?
> > > >
> > > >
> > > >
> > > > - I added some comments both inside and outside the code snippets in
> > KIP.
> > > > Specifically, for the code snippet above, we add *commit()* method to
> > > > *RecordContext* interface.
> > > > However, we want  *commit()* method to be used only for
> *RecordContext*
> > > > instances (at least for now), so we add UnsupportedOperationException
> > in
> > > > all classes/interfaces that extend/implement *RecordContext.*
> > > > In general, 1) we make RecordContext publicly available within
> > > > ProcessorContext,  2) initialize its instance within all required
> > > > Processors and 3) pass it as an argument to the related Rich
> interfaces
> > > > inside Processors.
> > > >
> > > >
> > > >
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com> wrote:
> > > >
> > > > > 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.
> > > > >
> > > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > > >                              final Aggregator<? super K, ? super V,
> > VR>
> > > > > adder,
> > > > >
> > > > > The adder doesn't need to be RichAggregator ?
> > > > >
> > > > >   public RecordContext recordContext() {
> > > > >     return this.recordContext();
> > > > >
> > > > > Can you explain a bit about the above implementation ?
> > > > >
> > > > >    void commit () {
> > > > >      throw new UnsupportedOperationException("commit() is not
> > > supported
> > > > in
> > > > > this context");
> > > > >
> > > > > Is the exception going to be replaced with real code in the PR ?
> > > > >
> > > > > Cheers
> > > > >
> > > > >
> > > > > On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> > je.karimov@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > 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 <
> > > je.karimov@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > 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 <
> damian.guy@gmail.com
> > >
> > > > > 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 Jeyhun Karimov <
> > je.karimov@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > 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 merge this KIP into the upcoming release, as KIP-159 is
> not
> > > > voted
> > > > > > yet
> > > > > > >> > (because we want both KIP-149 and KIP-159 to be as an
> "atomic"
> > > > > merge).
> > > > > > >> So
> > > > > > >> > I decided to wait until KIP-182 gets stable (there are some
> > > minor
> > > > > > >> updates
> > > > > > >> > AFAIK) and update the KIP accordingly. Please correct me if
> I
> > am
> > > > > wrong
> > > > > > >> or I
> > > > > > >> > misunderstood.
> > > > > > >> >
> > > > > > >> > Cheers,
> > > > > > >> > Jeyhun
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > > damian.guy@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >
> > > > > > >> > > +1
> > > > > > >> > >
> > > > > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > > >> 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
> would
> > > > still
> > > > > > >> need to
> > > > > > >> > > > update your proposal for people to comment and vote on.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > Guozhang
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > > yuzhihong@gmail.com
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > +1
> > > > > > >> > > > >
> > > > > > >> > > > > One interface is cleaner.
> > > > > > >> > > > >
> > > > > > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > > > > bbejeck@gmail.com
> > > > > > >
> > > > > > >> > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > +1 for me on collapsing the RichXXXX and
> > > ValueXXXXWithKey
> > > > > > >> > interfaces
> > > > > > >> > > > > into 1
> > > > > > >> > > > > > interface.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks,
> > > > > > >> > > > > > Bill
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > > > > >> > > je.karimov@gmail.com
> > > > > > >> > > > >
> > > > > > >> > > > > > 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 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
> > > > > > >> > > > > > this
> > > > > > >> > > > > > > would not be a problem.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Our initial idea was similar to :
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> > > > > implements
> > > > > > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > > >> > > > > > > ......
> > > > > > >> > > > > > > }
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > So, we check the type of object, whether it is
> > RichXXX
> > > > or
> > > > > > >> > > XXXWithKey
> > > > > > >> > > > > > inside
> > > > > > >> > > > > > > the called method and continue accordingly.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > If this is ok with the community, I would like to
> > > revert
> > > > > the
> > > > > > >> > > current
> > > > > > >> > > > > > design
> > > > > > >> > > > > > > to this again.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Cheers,
> > > > > > >> > > > > > > Jeyhun
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > > > > >> damian.guy@gmail.com
> > > > > > >> > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hi Jeyhun,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Thanks for sending out the update. I guess i was
> > > > > thinking
> > > > > > >> more
> > > > > > >> > > > along
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > lines of option 2 where we collapse the RichXXXX
> > and
> > > > > > >> > > > ValueXXXXWithKey
> > > > > > >> > > > > > 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?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Thanks,
> > > > > > >> > > > > > > > Damian
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > > > > >> > > je.karimov@gmail.com>
> > > > > > >> > > > > > > wrote:
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > > 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 free to comment on this.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > [1]
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > https://cwiki.apache.org/
> > confluence/pages/viewpage.
> > > > > > >> > > > > > > action?pageId=73637757
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Cheers,
> > > > > > >> > > > > > > > > Jeyhun
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun
> Karimov <
> > > > > > >> > > > > je.karimov@gmail.com
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > > wrote:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > > 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 <
> > > > > > >> > > > > > > matthias@confluent.io>
> > > > > > >> > > > > > > > > > wrote:
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > > >> 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
> > > > > > >> > > > > > > > > >> > for one-shot solution for overloads
> issue.
> > > > > > >> > > > > > > > > >> >
> > > > > > >> > > > > > > > > >> > Cheers,
> > > > > > >> > > > > > > > > >> > Jeyhun
> > > > > > >> > > > > > > > > >> >
> > > > > > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian
> Guy
> > <
> > > > > > >> > > > > > damian.guy@gmail.com>
> > > > > > >> > > > > > > > > >> wrote:
> > > > > > >> > > > > > > > > >> >
> > > > > > >> > > > > > > > > >> >> 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
> > > > > > >> > > > > > > > > overrides.
> > > > > > >> > > > > > > > > >> >>
> > > > > > >> > > > > > > > > >> >> Thanks,
> > > > > > >> > > > > > > > > >> >> Damian
> > > > > > >> > > > > > > > > >> >>
> > > > > > >> > > > > > > > > >> >>
> > > > > > >> > > > > > > > > >> >>> About ProcessorContext vs
> RecordContext,
> > > you
> > > > > are
> > > > > > >> > right.
> > > > > > >> > > I
> > > > > > >> > > > > > think
> > > > > > >> > > > > > > I
> > > > > > >> > > > > > > > > >> need to
> > > > > > >> > > > > > > > > >> >>> implement a prototype to understand the
> > > full
> > > > > > >> picture
> > > > > > >> > as
> > > > > > >> > > > some
> > > > > > >> > > > > > > parts
> > > > > > >> > > > > > > > > of
> > > > > > >> > > > > > > > > >> the
> > > > > > >> > > > > > > > > >> >>> KIP might not be as straightforward as
> I
> > > > > thought.
> > > > > > >> > > > > > > > > >> >>>
> > > > > > >> > > > > > > > > >> >>>
> > > > > > >> > > > > > > > > >> >>> Cheers,
> > > > > > >> > > > > > > > > >> >>> Jeyhun
> > > > > > >> > > > > > > > > >> >>>
> > > > > > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian
> > Guy
> > > <
> > > > > > >> > > > > > > damian.guy@gmail.com>
> > > > > > >> > > > > > > > > >> wrote:
> > > > > > >> > > > > > > > > >> >>>
> > > > > > >> > > > > > > > > >> >>>> 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
> > > > > > >> > > > > lifetime
> > > > > > >> > > > > > of
> > > > > > >> > > > > > > > the
> > > > > > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense
> to
> > > > cast a
> > > > > > >> > > > > > ProcessorContext
> > > > > > >> > > > > > > > to
> > > > > > >> > > > > > > > > a
> > > > > > >> > > > > > > > > >> >>>> RecordContext.
> > > > > > >> > > > > > > > > >> >>>> You mentioned above passing the
> > > > > > >> > > InternalProcessorContext
> > > > > > >> > > > to
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > > >> init()
> > > > > > >> > > > > > > > > >> >>>> calls. It is internal for a reason
> and i
> > > > think
> > > > > > it
> > > > > > >> > > should
> > > > > > >> > > > > > remain
> > > > > > >> > > > > > > > > that
> > > > > > >> > > > > > > > > >> >> way.
> > > > > > >> > > > > > > > > >> >>>> It might be better to move the
> > > > recordContext()
> > > > > > >> method
> > > > > > >> > > > from
> > > > > > >> > > > > > > > > >> >>>> InternalProcessorContext to
> > > > ProcessorContext.
> > > > > > >> > > > > > > > > >> >>>>
> > > > > > >> > > > > > > > > >> >>>> In the KIP you have an example
> showing:
> > > > > > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > > > > > processorContext);
> > > > > > >> > > > > > > > > >> >>>> But the interface is:
> > > > > > >> > > > > > > > > >> >>>> public interface RichValueMapper<V,
> VR>
> > {
> > > > > > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> > > > > RecordContext
> > > > > > >> > > > > > recordContext);
> > > > > > >> > > > > > > > > >> >>>> }
> > > > > > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides
> as
> > > > above
> > > > > > this
> > > > > > >> > > > wouldn't
> > > > > > >> > > > > > > make
> > > > > > >> > > > > > > > > >> sense.
> > > > > > >> > > > > > > > > >> >>>>
> > > > > > >> > > > > > > > > >> >>>> Thanks,
> > > > > > >> > > > > > > > > >> >>>> Damian
> > > > > > >> > > > > > > > > >> >>>>
> > > > > > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> > > Karimov <
> > > > > > >> > > > > > > je.karimov@gmail.com
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > >> >> wrote:
> > > > > > >> > > > > > > > > >> >>>>
> > > > > > >> > > > > > > > > >> >>>>> 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.
> > > > > > >> Can't
> > > > > > >> > we
> > > > > > >> > > > pass
> > > > > > >> > > > > > > this
> > > > > > >> > > > > > > > > as a
> > > > > > >> > > > > > > > > >> >>>>> parameter to init() method of
> > processors?
> > > > > Then
> > > > > > we
> > > > > > >> > > would
> > > > > > >> > > > be
> > > > > > >> > > > > > > able
> > > > > > >> > > > > > > > to
> > > > > > >> > > > > > > > > >> >> get
> > > > > > >> > > > > > > > > >> >>>>> RecordContext easily with just a
> method
> > > > call.
> > > > > > >> > > > > > > > > >> >>>>>
> > > > > > >> > > > > > > > > >> >>>>>
> > > > > > >> > > > > > > > > >> >>>>> Cheers,
> > > > > > >> > > > > > > > > >> >>>>> Jeyhun
> > > > > > >> > > > > > > > > >> >>>>>
> > > > > > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM
> > Matthias
> > > > J.
> > > > > > Sax
> > > > > > >> <
> > > > > > >> > > > > > > > > >> >>> matthias@confluent.io>
> > > > > > >> > > > > > > > > >> >>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>
> > > > > > >> > > > > > > > > >> >>>>>> 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
> > > > > > >> > > > > > > > > >> >>> will
> > > > > > >> > > > > > > > > >> >>>>>> be provided by the consecutive call
> to
> > > > > > aggregate
> > > > > > >> > > > anyways.
> > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > >> > > > > > > > > >> >>>>>> -Matthias
> > > > > > >> > > > > > > > > >> >>>>>>
> > > > > > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax
> > > wrote:
> > > > > > >> > > > > > > > > >> >>>>>>> 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
> > > > implement
> > > > > > >> both.
> > > > > > >> > > This
> > > > > > >> > > > > > allows
> > > > > > >> > > > > > > > us
> > > > > > >> > > > > > > > > >> >> to
> > > > > > >> > > > > > > > > >> >>>> cast
> > > > > > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus
> limit
> > > the
> > > > > > >> visible
> > > > > > >> > > > scope.
> > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>> -Matthias
> > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov
> > > wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>> 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 address this issue
> as
> > > well
> > > > > or
> > > > > > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>> Cheers,
> > > > > > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> > > > Guozhang
> > > > > > >> Wang <
> > > > > > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > > > > >> > > > > > > > > >> >>>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM,
> > > Jeyhun
> > > > > > >> Karimov
> > > > > > >> > <
> > > > > > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > >> > > > > > > > > >> >>>>>>>>> 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
> > KAFKA-3907
> > > > [1]
> > > > > > can
> > > > > > >> > also
> > > > > > >> > > > be
> > > > > > >> > > > > > > > > >> >> integrated
> > > > > > >> > > > > > > > > >> >>>> into
> > > > > > >> > > > > > > > > >> >>>>>> the
> > > > > > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method
> > inside
> > > > > > >> > RecordContext
> > > > > > >> > > > > > > > interface.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>> [1]
> > > > > > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > > > > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM
> > > > Matthias
> > > > > > J.
> > > > > > >> > Sax <
> > > > > > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> 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
> > > > > > information
> > > > > > >> > only.
> > > > > > >> > > > If
> > > > > > >> > > > > > you
> > > > > > >> > > > > > > > want
> > > > > > >> > > > > > > > > >> >> to
> > > > > > >> > > > > > > > > >> >>>> do
> > > > > > >> > > > > > > > > >> >>>>> a
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should
> > > > rather
> > > > > > use
> > > > > > >> > > > > > #transform().
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we
> > > would
> > > > > > need
> > > > > > >> to
> > > > > > >> > > > switch
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > a
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing
> many
> > > > more
> > > > > > >> > > overloads.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate
> for
> > a
> > > > > simple
> > > > > > >> > > > interface
> > > > > > >> > > > > > > with a
> > > > > > >> > > > > > > > > >> >>> single
> > > > > > >> > > > > > > > > >> >>>>>>>>> method
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext
> > > > object.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang
> Wang
> > > > wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > > > > > >> > > <
> > > > > > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I
> will
> > > not
> > > > > > >> mention
> > > > > > >> > > > them.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are
> > > spread
> > > > in
> > > > > > >> > KIP-149
> > > > > > >> > > > and
> > > > > > >> > > > > in
> > > > > > >> > > > > > > > this
> > > > > > >> > > > > > > > > >> >> KIP
> > > > > > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions
> > > came
> > > > > into
> > > > > > >> the
> > > > > > >> > > > stage
> > > > > > >> > > > > > with
> > > > > > >> > > > > > > > > >> >>> KIP-149,
> > > > > > >> > > > > > > > > >> >>>>> in
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a
> result
> > we
> > > > > > >> extended
> > > > > > >> > > > KIP-149
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > > >> >> support
> > > > > > >> > > > > > > > > >> >>>>> Rich
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich
> > > > functions,
> > > > > > we
> > > > > > >> > > > provided
> > > > > > >> > > > > > init
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian
> > > > suggested
> > > > > > >> that
> > > > > > >> > we
> > > > > > >> > > > > should
> > > > > > >> > > > > > > not
> > > > > > >> > > > > > > > > >> >>>> provide
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users.
> As a
> > > > > result,
> > > > > > >> we
> > > > > > >> > > > > separated
> > > > > > >> > > > > > > the
> > > > > > >> > > > > > > > > >> >> two
> > > > > > >> > > > > > > > > >> >>>>>>>>> problems
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> into
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it
> seems
> > > they
> > > > > can
> > > > > > >> be
> > > > > > >> > > > solved
> > > > > > >> > > > > in
> > > > > > >> > > > > > > > > >> >>> parallel.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered
> > was
> > > :
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > > > ValueMapperWithKey<K,
> > > > > > V,
> > > > > > >> > VR>
> > > > > > >> > > {
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key,
> > final V
> > > > > > value);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > RichValueMapper<K,
> > > > > V,
> > > > > > >> VR>
> > > > > > >> > > > extends
> > > > > > >> > > > > > > > > >> >>>> RichFunction{
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> RichFunction {
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > > > > > >> recordContext);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> RecordContext
> > {
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > appConfigs();
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > > > >> > > > > appConfigsWithPrefix(String
> > > > > > >> > > > > > > > > >> >> prefix);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > ProcessorContext
> > > > > > extends
> > > > > > >> > > > > > > RecordContext
> > > > > > >> > > > > > > > {
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the
> ones
> > in
> > > > > > >> > > RecordContext
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and
> > > "withoutKey"
> > > > > > >> > interfaces
> > > > > > >> > > > can
> > > > > > >> > > > > be
> > > > > > >> > > > > > > > > >> >>> converted
> > > > > > >> > > > > > > > > >> >>>> to
> > > > > > >> > > > > > > > > >> >>>>>>>>>> their
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty
> > > > init()
> > > > > > and
> > > > > > >> > > close()
> > > > > > >> > > > > > > > methods)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors
> will
> > > > accept
> > > > > > >> Rich
> > > > > > >> > > > > > interfaces
> > > > > > >> > > > > > > in
> > > > > > >> > > > > > > > > >> >>> their
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related
> > > > > "withKey"
> > > > > > >> or
> > > > > > >> > > > > > > "withoutKey"
> > > > > > >> > > > > > > > > >> >>>>> interfaces
> > > > > > >> > > > > > > > > >> >>>>>>>>> to
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> Rich
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the
> > > > topology
> > > > > > and
> > > > > > >> > > > > initialize
> > > > > > >> > > > > > > the
> > > > > > >> > > > > > > > > >> >>>> related
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> processors
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to
> > overloaded
> > > > > > methods
> > > > > > >> > for
> > > > > > >> > > > rich
> > > > > > >> > > > > > > > > >> >> functions
> > > > > > >> > > > > > > > > >> >>> as
> > > > > > >> > > > > > > > > >> >>>>>> Rich
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey
> > > > interfaces.
> > > > > > We
> > > > > > >> > will
> > > > > > >> > > > just
> > > > > > >> > > > > > > check
> > > > > > >> > > > > > > > > >> >> the
> > > > > > >> > > > > > > > > >> >>>>> object
> > > > > > >> > > > > > > > > >> >>>>>>>>>> type
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts
> that
> > > the
> > > > > > above
> > > > > > >> > > > approach
> > > > > > >> > > > > > > does
> > > > > > >> > > > > > > > > not
> > > > > > >> > > > > > > > > >> >>>>> support
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> lambdas
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one
> > > > method,
> > > > > > >> only
> > > > > > >> > > > > > > > > >> >>> init(RecordContext),
> > > > > > >> > > > > > > > > >> >>>>> as
> > > > > > >> > > > > > > > > >> >>>>>>>>>> part
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> of
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> > > > > > Personally I
> > > > > > >> > > think
> > > > > > >> > > > > Rich
> > > > > > >> > > > > > > > > >> >>> interfaces
> > > > > > >> > > > > > > > > >> >>>>> are
> > > > > > >> > > > > > > > > >> >>>>>>>>> by
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we
> > > > should
> > > > > > not
> > > > > > >> > care
> > > > > > >> > > > much
> > > > > > >> > > > > > > about
> > > > > > >> > > > > > > > > >> >> it.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's
> > > discussion,
> > > > > an
> > > > > > >> > > > alternative
> > > > > > >> > > > > > we
> > > > > > >> > > > > > > > > >> >>>> considered
> > > > > > >> > > > > > > > > >> >>>>>> was
> > > > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as
> > > method
> > > > > > >> > parameter.
> > > > > > >> > > > > This
> > > > > > >> > > > > > > > might
> > > > > > >> > > > > > > > > >> >>> even
> > > > > > >> > > > > > > > > >> >>>>>>>>> allow
> > > > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep
> > the
> > > > > name
> > > > > > >> > > > > RichFunction
> > > > > > >> > > > > > as
> > > > > > >> > > > > > > > we
> > > > > > >> > > > > > > > > >> >>>>> preserve
> > > > > > >> > > > > > > > > >> >>>>>>>>> the
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and
> > > > > `close()`
> > > > > > we
> > > > > > >> > > > > basically
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an
> > in-memory
> > > > > state
> > > > > > >> for a
> > > > > > >> > > > > > function.
> > > > > > >> > > > > > > > > >> >> Thus,
> > > > > > >> > > > > > > > > >> >>> we
> > > > > > >> > > > > > > > > >> >>>>>>>>> cannot
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> > > > > > >> RichValueMapper
> > > > > > >> > > (etc)
> > > > > > >> > > > > > over
> > > > > > >> > > > > > > > > >> >>> multiple
> > > > > > >> > > > > > > > > >> >>>>>> tasks
> > > > > > >> > > > > > > > > >> >>>>>>>>>> and
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier
> > pattern
> > > > > > similar
> > > > > > >> to
> > > > > > >> > > > > > > > #transform().
> > > > > > >> > > > > > > > > >> >> And
> > > > > > >> > > > > > > > > >> >>>>> this
> > > > > > >> > > > > > > > > >> >>>>>>>>>> would
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API,
> as
> > > > > > >> > > > > > > > (Rich)ValueMapperSupplier
> > > > > > >> > > > > > > > > >> >>> would
> > > > > > >> > > > > > > > > >> >>>>> not
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and
> > thus
> > > > we
> > > > > > >> would
> > > > > > >> > > need
> > > > > > >> > > > > many
> > > > > > >> > > > > > > new
> > > > > > >> > > > > > > > > >> >>>> overload
> > > > > > >> > > > > > > > > >> >>>>>>>>> for
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes".
> (Copy
> > > > paste
> > > > > > from
> > > > > > >> > > > > Matthias's
> > > > > > >> > > > > > > > > email)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18
> AM
> > > > > Matthias
> > > > > > >> J.
> > > > > > >> > > Sax <
> > > > > > >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > > > > > >> > > > 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:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> interface
> RichValueMapper<V,
> > > VR>
> > > > {
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value,
> final
> > > > > > >> > RecordContext
> > > > > > >> > > > > > > context);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > > >> > mapValues(ValueMapper<?
> > > > > > >> > > > > super
> > > > > > >> > > > > > > V, ?
> > > > > > >> > > > > > > > > >> >>>> extends
> > > > > > >> > > > > > > > > >> >>>>>> VR>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > > >> > > > > > > > mapValueswithContext(RichValueMapper
> > > > > > >> > > > > > > > > >> >> <?
> > > > > > >> > > > > > > > > >> >>>>> super
> > > > > > >> > > > > > > > > >> >>>>>>>>>> V, ?
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > ------------------------------
> > > -
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will
> > > > > introduces
> > > > > > >> more
> > > > > > >> > > > > > > overloads;
> > > > > > >> > > > > > > > > >> >> but I
> > > > > > >> > > > > > > > > >> >>>>> think
> > > > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly
> > > introduced
> > > > > by
> > > > > > 1)
> > > > > > >> > > serde
> > > > > > >> > > > > > > > overrides
> > > > > > >> > > > > > > > > >> >> and
> > > > > > >> > > > > > > > > >> >>>> 2)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier
> > overides,
> > > > both
> > > > > > of
> > > > > > >> > which
> > > > > > >> > > > can
> > > > > > >> > > > > > be
> > > > > > >> > > > > > > > > >> >> reduced
> > > > > > >> > > > > > > > > >> >>>> in
> > > > > > >> > > > > > > > > >> >>>>>> the
> > > > > > >> > > > > > > > > >> >>>>>>>>>>> near
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this
> > > > overloading
> > > > > > is
> > > > > > >> > still
> > > > > > >> > > > > > > > worthwhile,
> > > > > > >> > > > > > > > > >> >> as
> > > > > > >> > > > > > > > > >> >>>> it
> > > > > > >> > > > > > > > > >> >>>>>> has
> > > > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda
> > > > expressions.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do
> not
> > > need
> > > > > to
> > > > > > >> > > "convert"
> > > > > > >> > > > > > from
> > > > > > >> > > > > > > > > >> >>> non-rich
> > > > > > >> > > > > > > > > >> >>>>>>>>>> functions
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> to
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has
> > already
> > > > > been
> > > > > > >> > > discussed
> > > > > > >> > > > > > and I
> > > > > > >> > > > > > > > may
> > > > > > >> > > > > > > > > >> >>> have
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> in
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways,
> > lmk.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at
> 10:18
> > > PM,
> > > > > > >> Matthias
> > > > > > >> > J.
> > > > > > >> > > > > Sax <
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture
> to
> > > > > > converge
> > > > > > >> to
> > > > > > >> > an
> > > > > > >> > > > > > overall
> > > > > > >> > > > > > > > > >> >>>> consisted
> > > > > > >> > > > > > > > > >> >>>>>>>>> API.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the
> > overloads.
> > > > Yes,
> > > > > > we
> > > > > > >> > might
> > > > > > >> > > > get
> > > > > > >> > > > > > > more
> > > > > > >> > > > > > > > > >> >>>> overload.
> > > > > > >> > > > > > > > > >> >>>>>> It
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> might
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to
> do
> > a
> > > > > single
> > > > > > >> > > > > > > xxxWithContext()
> > > > > > >> > > > > > > > > >> >>>> overload
> > > > > > >> > > > > > > > > >> >>>>>>>>> that
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> will
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> > > > > > Otherwise,
> > > > > > >> if
> > > > > > >> > > > might
> > > > > > >> > > > > > get
> > > > > > >> > > > > > > > too
> > > > > > >> > > > > > > > > >> >>> messy
> > > > > > >> > > > > > > > > >> >>>>>>>>> having
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper,
> > > ValueMapperWithKey,
> > > > > > >> > > > > > > > > ValueMapperWithContext,
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > ValueMapperWithKeyWithContext.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also
> > > have
> > > > > the
> > > > > > >> > > "builder
> > > > > > >> > > > > > > pattern"
> > > > > > >> > > > > > > > > >> >> idea
> > > > > > >> > > > > > > > > >> >>>> as
> > > > > > >> > > > > > > > > >> >>>>> an
> > > > > > >> > > > > > > > > >> >>>>>>>>>> API
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might
> > mitigate
> > > > the
> > > > > > >> > overload
> > > > > > >> > > > > > > problem.
> > > > > > >> > > > > > > > > Not
> > > > > > >> > > > > > > > > >> >>> for
> > > > > > >> > > > > > > > > >> >>>>>>>>> simple
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap
> > etc
> > > > but
> > > > > > for
> > > > > > >> > joins
> > > > > > >> > > > and
> > > > > > >> > > > > > > > > >> >>>> aggregations.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I
> > > > mentioned
> > > > > in
> > > > > > >> an
> > > > > > >> > > older
> > > > > > >> > > > > > > email,
> > > > > > >> > > > > > > > I
> > > > > > >> > > > > > > > > >> >> am
> > > > > > >> > > > > > > > > >> >>>>>>>>>> personally
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure
> > > > functional
> > > > > > >> > > interface,
> > > > > > >> > > > > and
> > > > > > >> > > > > > > add
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > > WithRecordContext
> > > > > with
> > > > > > >> > method
> > > > > > >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > > > > > >> > > > > > > > > >> >>>>>>>>>> (or
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better
> > > name)
> > > > > --
> > > > > > >> but
> > > > > > >> > not
> > > > > > >> > > > > > > > `close()`)
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > > > > > >> > ValueMapperWithRecordContext
> > > > > > >> > > > > > extends
> > > > > > >> > > > > > > > > >> >>>>> ValueMapper,
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to
> avoid
> > > any
> > > > > > >> > overload.
> > > > > > >> > > Of
> > > > > > >> > > > > > > course,
> > > > > > >> > > > > > > > > we
> > > > > > >> > > > > > > > > >> >>>> don't
> > > > > > >> > > > > > > > > >> >>>>>>>>> get
> > > > > > >> > > > > > > > > >> >>>>>>>>>> a
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface
> > and
> > > > also
> > > > > > >> > > sacrifices
> > > > > > >> > > > > > > > Lambdas.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little
> bit
> > > > > > undecided
> > > > > > >> > what
> > > > > > >> > > > the
> > > > > > >> > > > > > > > better
> > > > > > >> > > > > > > > > >> >>>> option
> > > > > > >> > > > > > > > > >> >>>>>>>>> might
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>> be.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other
> > > think
> > > > > > about
> > > > > > >> > this
> > > > > > >> > > > > trade
> > > > > > >> > > > > > > > off.
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun
> > > > Karimov
> > > > > > >> wrote:<
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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<V1, V2, VR, K> {
>     VR apply(final V1 value1, final V2 value2, final K key, final
> RecordContext
> recordContext);
> }



- Fixed.


2. Some of the listed functions are not necessary since their pairing APIs
> are being deprecated in 1.0 already:
> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ?
> super V, KR> selector,
>                                    final Serde<KR> keySerde,
>                                    final Serde<V> valSerde);
> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>                                  final RichValueJoiner<? super K, ? super
> V,
> ? super VT, ? extends VR> joiner,
>                                  final Serde<K> keySerde,
>                                  final Serde<V> valSerde);


-Fixed

3. For a few functions where we are adding three APIs for a combo of both
> mapper / joiner, or both initializer / aggregator, or adder / subtractor,
> I'm wondering if we can just keep one that use "rich" functions for both;
> so that we can have less overloads and let users who only want to access
> one of them to just use dummy parameter declarations. For example:
>
> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
>                                  final RichKeyValueMapper<? super K, ?
> super
>  V, ? extends GK> keyValueMapper,
>                                  final RichValueJoiner<? super K, ? super
> V,
> ? super GV, ? extends RV> joiner);



-Agreed. Fixed.


4. For TimeWindowedKStream, I'm wondering why we do not make its
> Initializer also "rich" functions? I.e.


- It was a typo. Fixed.


5. We need to move "RecordContext" from o.a.k.processor.internals to
> o.a.k.processor.
>
> 6. I'm not clear why we want to move `commit()` from ProcessorContext to
> RecordContext?
>

-
Because it makes sense logically and  to reduce code maintenance (both
interfaces have offset() timestamp() topic() partition() methods),  I
inherit ProcessorContext from RecordContext.
Since we need commit() method both in ProcessorContext and in RecordContext
I move commit() method to parent class (RecordContext).


Cheers,
Jeyhun



On Wed, Oct 11, 2017 at 12:59 AM, Guozhang Wang <wa...@gmail.com> wrote:

> 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<V1, V2, VR, K> {
>     VR apply(final V1 value1, final V2 value2, final K key, final
> RecordContext
> recordContext);
> }
>
> My motivation is that for lambda expression in J8, users that would not
> care about the key but only the context, or vice versa, is likely to write
> it as (value1, value2, dummy, context) -> ... than putting the dummy at the
> beginning of the parameter list. Generally speaking we'd like to make all
> the "necessary" parameters prior to optional ones.
>
>
> 2. Some of the listed functions are not necessary since their pairing APIs
> are being deprecated in 1.0 already:
>
> <KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ?
> super V, KR> selector,
>                                    final Serde<KR> keySerde,
>                                    final Serde<V> valSerde);
>
> <VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
>                                  final RichValueJoiner<? super K, ? super
> V,
> ? super VT, ? extends VR> joiner,
>                                  final Serde<K> keySerde,
>                                  final Serde<V> valSerde);
>
>
>
> 3. For a few functions where we are adding three APIs for a combo of both
> mapper / joiner, or both initializer / aggregator, or adder / subtractor,
> I'm wondering if we can just keep one that use "rich" functions for both;
> so that we can have less overloads and let users who only want to access
> one of them to just use dummy parameter declarations. For example:
>
>
> <GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
>                                  final RichKeyValueMapper<? super K, ?
> super
>  V, ? extends GK> keyValueMapper,
>                                  final RichValueJoiner<? super K, ? super
> V,
> ? super GV, ? extends RV> joiner);
>
> <VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
>                              final RichAggregator<? super K, ? super V, VR>
> aggregator,
>                              final Materialized<K, VR, KeyValueStore<Bytes,
> byte[]>> materialized);
>
> Similarly for KGroupedTable, a bunch of aggregate() are deprecated so we do
> not need to add its rich functions any more.
>
>
> 4. For TimeWindowedKStream, I'm wondering why we do not make its
> Initializer also "rich" functions? I.e.
>
> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> initializer,
>                                        final RichAggregator<? super K, ?
> super V, VR> aggregator);
> <VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
> initializer,
>                                        final RichAggregator<? super K, ?
> super V, VR> aggregator,
>                                        final Materialized<K, VR,
> WindowStore<Bytes, byte[]>> materialized);
>
>
> 5. We need to move "RecordContext" from o.a.k.processor.internals to
> o.a.k.processor.
>
> 6. I'm not clear why we want to move `commit()` from ProcessorContext to
> RecordContext? Conceptually I think it would better staying in the
> ProcessorContext. Do you find this not doable in the internal
> implementations?
>
>
> Guozhang
>
>
>
> On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:
>
> >    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 <je...@gmail.com>
> > wrote:
> >
> > > 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 to KIP.
> > >
> > >
> > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > >                              final Aggregator<? super K, ? super V,
> VR>
> > > > adder,
> > > > The adder doesn't need to be RichAggregator ?
> > >
> > >
> > >
> > > - Exactly. However, there are 2 Aggregator-type arguments in the
> related
> > > method. So, I had to overload all possible their Rich counterparts:
> > >
> > > // adder with non-rich, subtrctor is rich
> > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >                              final Aggregator<? super K, ? super V, VR>
> > > adder,
> > >                              final RichAggregator<? super K, ? super V,
> > VR>
> > > subtractor,
> > >                              final Materialized<K, VR,
> > KeyValueStore<Bytes,
> > > byte[]>> materialized);
> > >
> > > // adder withrich, subtrctor is non-rich
> > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >                              final RichAggregator<? super K, ? super V,
> > VR>
> > > adder,
> > >                              final Aggregator<? super K, ? super V, VR>
> > > subtractor,
> > >                              final Materialized<K, VR,
> > KeyValueStore<Bytes,
> > > byte[]>> materialized);
> > >
> > > // both adder and subtractor are rich
> > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >                              final RichAggregator<? super K, ? super V,
> > VR>
> > > adder,
> > >                              final RichAggregator<? super K, ? super V,
> > VR>
> > > subtractor,
> > >                              final Materialized<K, VR,
> > KeyValueStore<Bytes,
> > > byte[]>> materialized);
> > >
> > >
> > > Can you explain a bit about the above implementation ?
> > > >    void commit () {
> > > >      throw new UnsupportedOperationException("commit() is not
> > supported
> > > in
> > > > this context");
> > > > Is the exception going to be replaced with real code in the PR ?
> > >
> > >
> > >
> > > - I added some comments both inside and outside the code snippets in
> KIP.
> > > Specifically, for the code snippet above, we add *commit()* method to
> > > *RecordContext* interface.
> > > However, we want  *commit()* method to be used only for *RecordContext*
> > > instances (at least for now), so we add UnsupportedOperationException
> in
> > > all classes/interfaces that extend/implement *RecordContext.*
> > > In general, 1) we make RecordContext publicly available within
> > > ProcessorContext,  2) initialize its instance within all required
> > > Processors and 3) pass it as an argument to the related Rich interfaces
> > > inside Processors.
> > >
> > >
> > >
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com> wrote:
> > >
> > > > 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.
> > > >
> > > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > > >                              final Aggregator<? super K, ? super V,
> VR>
> > > > adder,
> > > >
> > > > The adder doesn't need to be RichAggregator ?
> > > >
> > > >   public RecordContext recordContext() {
> > > >     return this.recordContext();
> > > >
> > > > Can you explain a bit about the above implementation ?
> > > >
> > > >    void commit () {
> > > >      throw new UnsupportedOperationException("commit() is not
> > supported
> > > in
> > > > this context");
> > > >
> > > > Is the exception going to be replaced with real code in the PR ?
> > > >
> > > > Cheers
> > > >
> > > >
> > > > On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <
> je.karimov@gmail.com>
> > > > wrote:
> > > >
> > > > > 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 <
> > je.karimov@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > 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 <damian.guy@gmail.com
> >
> > > > 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 Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > > > > wrote:
> > > > > >>
> > > > > >> > 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 merge this KIP into the upcoming release, as KIP-159 is not
> > > voted
> > > > > yet
> > > > > >> > (because we want both KIP-149 and KIP-159 to be as an "atomic"
> > > > merge).
> > > > > >> So
> > > > > >> > I decided to wait until KIP-182 gets stable (there are some
> > minor
> > > > > >> updates
> > > > > >> > AFAIK) and update the KIP accordingly. Please correct me if I
> am
> > > > wrong
> > > > > >> or I
> > > > > >> > misunderstood.
> > > > > >> >
> > > > > >> > Cheers,
> > > > > >> > Jeyhun
> > > > > >> >
> > > > > >> >
> > > > > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> > damian.guy@gmail.com>
> > > > > >> wrote:
> > > > > >> >
> > > > > >> > > +1
> > > > > >> > >
> > > > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > >> 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 would
> > > still
> > > > > >> need to
> > > > > >> > > > update your proposal for people to comment and vote on.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > Guozhang
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> > yuzhihong@gmail.com
> > > >
> > > > > >> wrote:
> > > > > >> > > >
> > > > > >> > > > > +1
> > > > > >> > > > >
> > > > > >> > > > > One interface is cleaner.
> > > > > >> > > > >
> > > > > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > > > bbejeck@gmail.com
> > > > > >
> > > > > >> > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > +1 for me on collapsing the RichXXXX and
> > ValueXXXXWithKey
> > > > > >> > interfaces
> > > > > >> > > > > into 1
> > > > > >> > > > > > interface.
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks,
> > > > > >> > > > > > Bill
> > > > > >> > > > > >
> > > > > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > > > >> > > je.karimov@gmail.com
> > > > > >> > > > >
> > > > > >> > > > > > 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 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
> > > > > >> > > > > > this
> > > > > >> > > > > > > would not be a problem.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Our initial idea was similar to :
> > > > > >> > > > > > >
> > > > > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> > > > implements
> > > > > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > >> > > > > > > ......
> > > > > >> > > > > > > }
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > So, we check the type of object, whether it is
> RichXXX
> > > or
> > > > > >> > > XXXWithKey
> > > > > >> > > > > > inside
> > > > > >> > > > > > > the called method and continue accordingly.
> > > > > >> > > > > > >
> > > > > >> > > > > > > If this is ok with the community, I would like to
> > revert
> > > > the
> > > > > >> > > current
> > > > > >> > > > > > design
> > > > > >> > > > > > > to this again.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Cheers,
> > > > > >> > > > > > > Jeyhun
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > > > >> damian.guy@gmail.com
> > > > > >> > >
> > > > > >> > > > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hi Jeyhun,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Thanks for sending out the update. I guess i was
> > > > thinking
> > > > > >> more
> > > > > >> > > > along
> > > > > >> > > > > > the
> > > > > >> > > > > > > > lines of option 2 where we collapse the RichXXXX
> and
> > > > > >> > > > ValueXXXXWithKey
> > > > > >> > > > > > 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?
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Thanks,
> > > > > >> > > > > > > > Damian
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > > > >> > > je.karimov@gmail.com>
> > > > > >> > > > > > > wrote:
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > > 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 free to comment on this.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > [1]
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > https://cwiki.apache.org/
> confluence/pages/viewpage.
> > > > > >> > > > > > > action?pageId=73637757
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Cheers,
> > > > > >> > > > > > > > > Jeyhun
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > > > >> > > > > je.karimov@gmail.com
> > > > > >> > > > > > >
> > > > > >> > > > > > > > > wrote:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > > 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 <
> > > > > >> > > > > > > matthias@confluent.io>
> > > > > >> > > > > > > > > > wrote:
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > > >> 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
> > > > > >> > > > > > > > > >> > for one-shot solution for overloads issue.
> > > > > >> > > > > > > > > >> >
> > > > > >> > > > > > > > > >> > Cheers,
> > > > > >> > > > > > > > > >> > Jeyhun
> > > > > >> > > > > > > > > >> >
> > > > > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy
> <
> > > > > >> > > > > > damian.guy@gmail.com>
> > > > > >> > > > > > > > > >> wrote:
> > > > > >> > > > > > > > > >> >
> > > > > >> > > > > > > > > >> >> 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
> > > > > >> > > > > > > > > overrides.
> > > > > >> > > > > > > > > >> >>
> > > > > >> > > > > > > > > >> >> Thanks,
> > > > > >> > > > > > > > > >> >> Damian
> > > > > >> > > > > > > > > >> >>
> > > > > >> > > > > > > > > >> >>
> > > > > >> > > > > > > > > >> >>> About ProcessorContext vs RecordContext,
> > you
> > > > are
> > > > > >> > right.
> > > > > >> > > I
> > > > > >> > > > > > think
> > > > > >> > > > > > > I
> > > > > >> > > > > > > > > >> need to
> > > > > >> > > > > > > > > >> >>> implement a prototype to understand the
> > full
> > > > > >> picture
> > > > > >> > as
> > > > > >> > > > some
> > > > > >> > > > > > > parts
> > > > > >> > > > > > > > > of
> > > > > >> > > > > > > > > >> the
> > > > > >> > > > > > > > > >> >>> KIP might not be as straightforward as I
> > > > thought.
> > > > > >> > > > > > > > > >> >>>
> > > > > >> > > > > > > > > >> >>>
> > > > > >> > > > > > > > > >> >>> Cheers,
> > > > > >> > > > > > > > > >> >>> Jeyhun
> > > > > >> > > > > > > > > >> >>>
> > > > > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian
> Guy
> > <
> > > > > >> > > > > > > damian.guy@gmail.com>
> > > > > >> > > > > > > > > >> wrote:
> > > > > >> > > > > > > > > >> >>>
> > > > > >> > > > > > > > > >> >>>> 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
> > > > > >> > > > > lifetime
> > > > > >> > > > > > of
> > > > > >> > > > > > > > the
> > > > > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to
> > > cast a
> > > > > >> > > > > > ProcessorContext
> > > > > >> > > > > > > > to
> > > > > >> > > > > > > > > a
> > > > > >> > > > > > > > > >> >>>> RecordContext.
> > > > > >> > > > > > > > > >> >>>> You mentioned above passing the
> > > > > >> > > InternalProcessorContext
> > > > > >> > > > to
> > > > > >> > > > > > the
> > > > > >> > > > > > > > > >> init()
> > > > > >> > > > > > > > > >> >>>> calls. It is internal for a reason and i
> > > think
> > > > > it
> > > > > >> > > should
> > > > > >> > > > > > remain
> > > > > >> > > > > > > > > that
> > > > > >> > > > > > > > > >> >> way.
> > > > > >> > > > > > > > > >> >>>> It might be better to move the
> > > recordContext()
> > > > > >> method
> > > > > >> > > > from
> > > > > >> > > > > > > > > >> >>>> InternalProcessorContext to
> > > ProcessorContext.
> > > > > >> > > > > > > > > >> >>>>
> > > > > >> > > > > > > > > >> >>>> In the KIP you have an example showing:
> > > > > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > > > > processorContext);
> > > > > >> > > > > > > > > >> >>>> But the interface is:
> > > > > >> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR>
> {
> > > > > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> > > > RecordContext
> > > > > >> > > > > > recordContext);
> > > > > >> > > > > > > > > >> >>>> }
> > > > > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides as
> > > above
> > > > > this
> > > > > >> > > > wouldn't
> > > > > >> > > > > > > make
> > > > > >> > > > > > > > > >> sense.
> > > > > >> > > > > > > > > >> >>>>
> > > > > >> > > > > > > > > >> >>>> Thanks,
> > > > > >> > > > > > > > > >> >>>> Damian
> > > > > >> > > > > > > > > >> >>>>
> > > > > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> > Karimov <
> > > > > >> > > > > > > je.karimov@gmail.com
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > >> >> wrote:
> > > > > >> > > > > > > > > >> >>>>
> > > > > >> > > > > > > > > >> >>>>> 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.
> > > > > >> Can't
> > > > > >> > we
> > > > > >> > > > pass
> > > > > >> > > > > > > this
> > > > > >> > > > > > > > > as a
> > > > > >> > > > > > > > > >> >>>>> parameter to init() method of
> processors?
> > > > Then
> > > > > we
> > > > > >> > > would
> > > > > >> > > > be
> > > > > >> > > > > > > able
> > > > > >> > > > > > > > to
> > > > > >> > > > > > > > > >> >> get
> > > > > >> > > > > > > > > >> >>>>> RecordContext easily with just a method
> > > call.
> > > > > >> > > > > > > > > >> >>>>>
> > > > > >> > > > > > > > > >> >>>>>
> > > > > >> > > > > > > > > >> >>>>> Cheers,
> > > > > >> > > > > > > > > >> >>>>> Jeyhun
> > > > > >> > > > > > > > > >> >>>>>
> > > > > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM
> Matthias
> > > J.
> > > > > Sax
> > > > > >> <
> > > > > >> > > > > > > > > >> >>> matthias@confluent.io>
> > > > > >> > > > > > > > > >> >>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>
> > > > > >> > > > > > > > > >> >>>>>> 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
> > > > > >> > > > > > > > > >> >>> will
> > > > > >> > > > > > > > > >> >>>>>> be provided by the consecutive call to
> > > > > aggregate
> > > > > >> > > > anyways.
> > > > > >> > > > > > > > > >> >>>>>>
> > > > > >> > > > > > > > > >> >>>>>>
> > > > > >> > > > > > > > > >> >>>>>> -Matthias
> > > > > >> > > > > > > > > >> >>>>>>
> > > > > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax
> > wrote:
> > > > > >> > > > > > > > > >> >>>>>>> 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
> > > implement
> > > > > >> both.
> > > > > >> > > This
> > > > > >> > > > > > allows
> > > > > >> > > > > > > > us
> > > > > >> > > > > > > > > >> >> to
> > > > > >> > > > > > > > > >> >>>> cast
> > > > > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit
> > the
> > > > > >> visible
> > > > > >> > > > scope.
> > > > > >> > > > > > > > > >> >>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>> -Matthias
> > > > > >> > > > > > > > > >> >>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov
> > wrote:
> > > > > >> > > > > > > > > >> >>>>>>>> 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 address this issue as
> > well
> > > > or
> > > > > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>> Cheers,
> > > > > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> > > Guozhang
> > > > > >> Wang <
> > > > > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > > > >> > > > > > > > > >> >>>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > > > > >> > > > > > > > > >> >>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM,
> > Jeyhun
> > > > > >> Karimov
> > > > > >> > <
> > > > > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > >> > > > > > > > > >> >>>>>>>>> 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
> KAFKA-3907
> > > [1]
> > > > > can
> > > > > >> > also
> > > > > >> > > > be
> > > > > >> > > > > > > > > >> >> integrated
> > > > > >> > > > > > > > > >> >>>> into
> > > > > >> > > > > > > > > >> >>>>>> the
> > > > > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method
> inside
> > > > > >> > RecordContext
> > > > > >> > > > > > > > interface.
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>> [1]
> > > > > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > > > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM
> > > Matthias
> > > > > J.
> > > > > >> > Sax <
> > > > > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> 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
> > > > > information
> > > > > >> > only.
> > > > > >> > > > If
> > > > > >> > > > > > you
> > > > > >> > > > > > > > want
> > > > > >> > > > > > > > > >> >> to
> > > > > >> > > > > > > > > >> >>>> do
> > > > > >> > > > > > > > > >> >>>>> a
> > > > > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should
> > > rather
> > > > > use
> > > > > >> > > > > > #transform().
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we
> > would
> > > > > need
> > > > > >> to
> > > > > >> > > > switch
> > > > > >> > > > > > to
> > > > > >> > > > > > > a
> > > > > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many
> > > more
> > > > > >> > > overloads.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for
> a
> > > > simple
> > > > > >> > > > interface
> > > > > >> > > > > > > with a
> > > > > >> > > > > > > > > >> >>> single
> > > > > >> > > > > > > > > >> >>>>>>>>> method
> > > > > >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext
> > > object.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang
> > > wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > > > > >> > > <
> > > > > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will
> > not
> > > > > >> mention
> > > > > >> > > > them.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are
> > spread
> > > in
> > > > > >> > KIP-149
> > > > > >> > > > and
> > > > > >> > > > > in
> > > > > >> > > > > > > > this
> > > > > >> > > > > > > > > >> >> KIP
> > > > > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions
> > came
> > > > into
> > > > > >> the
> > > > > >> > > > stage
> > > > > >> > > > > > with
> > > > > >> > > > > > > > > >> >>> KIP-149,
> > > > > >> > > > > > > > > >> >>>>> in
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result
> we
> > > > > >> extended
> > > > > >> > > > KIP-149
> > > > > >> > > > > > to
> > > > > >> > > > > > > > > >> >> support
> > > > > >> > > > > > > > > >> >>>>> Rich
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich
> > > functions,
> > > > > we
> > > > > >> > > > provided
> > > > > >> > > > > > init
> > > > > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian
> > > suggested
> > > > > >> that
> > > > > >> > we
> > > > > >> > > > > should
> > > > > >> > > > > > > not
> > > > > >> > > > > > > > > >> >>>> provide
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a
> > > > result,
> > > > > >> we
> > > > > >> > > > > separated
> > > > > >> > > > > > > the
> > > > > >> > > > > > > > > >> >> two
> > > > > >> > > > > > > > > >> >>>>>>>>> problems
> > > > > >> > > > > > > > > >> >>>>>>>>>>> into
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems
> > they
> > > > can
> > > > > >> be
> > > > > >> > > > solved
> > > > > >> > > > > in
> > > > > >> > > > > > > > > >> >>> parallel.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered
> was
> > :
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > > ValueMapperWithKey<K,
> > > > > V,
> > > > > >> > VR>
> > > > > >> > > {
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key,
> final V
> > > > > value);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > RichValueMapper<K,
> > > > V,
> > > > > >> VR>
> > > > > >> > > > extends
> > > > > >> > > > > > > > > >> >>>> RichFunction{
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > > > > >> recordContext);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext
> {
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > appConfigs();
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > > >> > > > > appConfigsWithPrefix(String
> > > > > >> > > > > > > > > >> >> prefix);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> ProcessorContext
> > > > > extends
> > > > > >> > > > > > > RecordContext
> > > > > >> > > > > > > > {
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones
> in
> > > > > >> > > RecordContext
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and
> > "withoutKey"
> > > > > >> > interfaces
> > > > > >> > > > can
> > > > > >> > > > > be
> > > > > >> > > > > > > > > >> >>> converted
> > > > > >> > > > > > > > > >> >>>> to
> > > > > >> > > > > > > > > >> >>>>>>>>>> their
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty
> > > init()
> > > > > and
> > > > > >> > > close()
> > > > > >> > > > > > > > methods)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will
> > > accept
> > > > > >> Rich
> > > > > >> > > > > > interfaces
> > > > > >> > > > > > > in
> > > > > >> > > > > > > > > >> >>> their
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related
> > > > "withKey"
> > > > > >> or
> > > > > >> > > > > > > "withoutKey"
> > > > > >> > > > > > > > > >> >>>>> interfaces
> > > > > >> > > > > > > > > >> >>>>>>>>> to
> > > > > >> > > > > > > > > >> >>>>>>>>>>> Rich
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the
> > > topology
> > > > > and
> > > > > >> > > > > initialize
> > > > > >> > > > > > > the
> > > > > >> > > > > > > > > >> >>>> related
> > > > > >> > > > > > > > > >> >>>>>>>>>>> processors
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to
> overloaded
> > > > > methods
> > > > > >> > for
> > > > > >> > > > rich
> > > > > >> > > > > > > > > >> >> functions
> > > > > >> > > > > > > > > >> >>> as
> > > > > >> > > > > > > > > >> >>>>>> Rich
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey
> > > interfaces.
> > > > > We
> > > > > >> > will
> > > > > >> > > > just
> > > > > >> > > > > > > check
> > > > > >> > > > > > > > > >> >> the
> > > > > >> > > > > > > > > >> >>>>> object
> > > > > >> > > > > > > > > >> >>>>>>>>>> type
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that
> > the
> > > > > above
> > > > > >> > > > approach
> > > > > >> > > > > > > does
> > > > > >> > > > > > > > > not
> > > > > >> > > > > > > > > >> >>>>> support
> > > > > >> > > > > > > > > >> >>>>>>>>>>> lambdas
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one
> > > method,
> > > > > >> only
> > > > > >> > > > > > > > > >> >>> init(RecordContext),
> > > > > >> > > > > > > > > >> >>>>> as
> > > > > >> > > > > > > > > >> >>>>>>>>>> part
> > > > > >> > > > > > > > > >> >>>>>>>>>>> of
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> > > > > Personally I
> > > > > >> > > think
> > > > > >> > > > > Rich
> > > > > >> > > > > > > > > >> >>> interfaces
> > > > > >> > > > > > > > > >> >>>>> are
> > > > > >> > > > > > > > > >> >>>>>>>>> by
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we
> > > should
> > > > > not
> > > > > >> > care
> > > > > >> > > > much
> > > > > >> > > > > > > about
> > > > > >> > > > > > > > > >> >> it.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's
> > discussion,
> > > > an
> > > > > >> > > > alternative
> > > > > >> > > > > > we
> > > > > >> > > > > > > > > >> >>>> considered
> > > > > >> > > > > > > > > >> >>>>>> was
> > > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as
> > method
> > > > > >> > parameter.
> > > > > >> > > > > This
> > > > > >> > > > > > > > might
> > > > > >> > > > > > > > > >> >>> even
> > > > > >> > > > > > > > > >> >>>>>>>>> allow
> > > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep
> the
> > > > name
> > > > > >> > > > > RichFunction
> > > > > >> > > > > > as
> > > > > >> > > > > > > > we
> > > > > >> > > > > > > > > >> >>>>> preserve
> > > > > >> > > > > > > > > >> >>>>>>>>> the
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and
> > > > `close()`
> > > > > we
> > > > > >> > > > > basically
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an
> in-memory
> > > > state
> > > > > >> for a
> > > > > >> > > > > > function.
> > > > > >> > > > > > > > > >> >> Thus,
> > > > > >> > > > > > > > > >> >>> we
> > > > > >> > > > > > > > > >> >>>>>>>>> cannot
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> > > > > >> RichValueMapper
> > > > > >> > > (etc)
> > > > > >> > > > > > over
> > > > > >> > > > > > > > > >> >>> multiple
> > > > > >> > > > > > > > > >> >>>>>> tasks
> > > > > >> > > > > > > > > >> >>>>>>>>>> and
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier
> pattern
> > > > > similar
> > > > > >> to
> > > > > >> > > > > > > > #transform().
> > > > > >> > > > > > > > > >> >> And
> > > > > >> > > > > > > > > >> >>>>> this
> > > > > >> > > > > > > > > >> >>>>>>>>>> would
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > > >> > > > > > > > (Rich)ValueMapperSupplier
> > > > > >> > > > > > > > > >> >>> would
> > > > > >> > > > > > > > > >> >>>>> not
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and
> thus
> > > we
> > > > > >> would
> > > > > >> > > need
> > > > > >> > > > > many
> > > > > >> > > > > > > new
> > > > > >> > > > > > > > > >> >>>> overload
> > > > > >> > > > > > > > > >> >>>>>>>>> for
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy
> > > paste
> > > > > from
> > > > > >> > > > > Matthias's
> > > > > >> > > > > > > > > email)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM
> > > > Matthias
> > > > > >> J.
> > > > > >> > > Sax <
> > > > > >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > > > > >> > > > 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:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V,
> > VR>
> > > {
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> > > > > >> > RecordContext
> > > > > >> > > > > > > context);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > >> > mapValues(ValueMapper<?
> > > > > >> > > > > super
> > > > > >> > > > > > > V, ?
> > > > > >> > > > > > > > > >> >>>> extends
> > > > > >> > > > > > > > > >> >>>>>> VR>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > >> > > > > > > > mapValueswithContext(RichValueMapper
> > > > > >> > > > > > > > > >> >> <?
> > > > > >> > > > > > > > > >> >>>>> super
> > > > > >> > > > > > > > > >> >>>>>>>>>> V, ?
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> ------------------------------
> > -
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will
> > > > introduces
> > > > > >> more
> > > > > >> > > > > > > overloads;
> > > > > >> > > > > > > > > >> >> but I
> > > > > >> > > > > > > > > >> >>>>> think
> > > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly
> > introduced
> > > > by
> > > > > 1)
> > > > > >> > > serde
> > > > > >> > > > > > > > overrides
> > > > > >> > > > > > > > > >> >> and
> > > > > >> > > > > > > > > >> >>>> 2)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier
> overides,
> > > both
> > > > > of
> > > > > >> > which
> > > > > >> > > > can
> > > > > >> > > > > > be
> > > > > >> > > > > > > > > >> >> reduced
> > > > > >> > > > > > > > > >> >>>> in
> > > > > >> > > > > > > > > >> >>>>>> the
> > > > > >> > > > > > > > > >> >>>>>>>>>>> near
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this
> > > overloading
> > > > > is
> > > > > >> > still
> > > > > >> > > > > > > > worthwhile,
> > > > > >> > > > > > > > > >> >> as
> > > > > >> > > > > > > > > >> >>>> it
> > > > > >> > > > > > > > > >> >>>>>> has
> > > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda
> > > expressions.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not
> > need
> > > > to
> > > > > >> > > "convert"
> > > > > >> > > > > > from
> > > > > >> > > > > > > > > >> >>> non-rich
> > > > > >> > > > > > > > > >> >>>>>>>>>> functions
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> to
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has
> already
> > > > been
> > > > > >> > > discussed
> > > > > >> > > > > > and I
> > > > > >> > > > > > > > may
> > > > > >> > > > > > > > > >> >>> have
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>> in
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways,
> lmk.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18
> > PM,
> > > > > >> Matthias
> > > > > >> > J.
> > > > > >> > > > > Sax <
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to
> > > > > converge
> > > > > >> to
> > > > > >> > an
> > > > > >> > > > > > overall
> > > > > >> > > > > > > > > >> >>>> consisted
> > > > > >> > > > > > > > > >> >>>>>>>>> API.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the
> overloads.
> > > Yes,
> > > > > we
> > > > > >> > might
> > > > > >> > > > get
> > > > > >> > > > > > > more
> > > > > >> > > > > > > > > >> >>>> overload.
> > > > > >> > > > > > > > > >> >>>>>> It
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> might
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do
> a
> > > > single
> > > > > >> > > > > > > xxxWithContext()
> > > > > >> > > > > > > > > >> >>>> overload
> > > > > >> > > > > > > > > >> >>>>>>>>> that
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> will
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> > > > > Otherwise,
> > > > > >> if
> > > > > >> > > > might
> > > > > >> > > > > > get
> > > > > >> > > > > > > > too
> > > > > >> > > > > > > > > >> >>> messy
> > > > > >> > > > > > > > > >> >>>>>>>>> having
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper,
> > ValueMapperWithKey,
> > > > > >> > > > > > > > > ValueMapperWithContext,
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> ValueMapperWithKeyWithContext.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also
> > have
> > > > the
> > > > > >> > > "builder
> > > > > >> > > > > > > pattern"
> > > > > >> > > > > > > > > >> >> idea
> > > > > >> > > > > > > > > >> >>>> as
> > > > > >> > > > > > > > > >> >>>>> an
> > > > > >> > > > > > > > > >> >>>>>>>>>> API
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might
> mitigate
> > > the
> > > > > >> > overload
> > > > > >> > > > > > > problem.
> > > > > >> > > > > > > > > Not
> > > > > >> > > > > > > > > >> >>> for
> > > > > >> > > > > > > > > >> >>>>>>>>> simple
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap
> etc
> > > but
> > > > > for
> > > > > >> > joins
> > > > > >> > > > and
> > > > > >> > > > > > > > > >> >>>> aggregations.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I
> > > mentioned
> > > > in
> > > > > >> an
> > > > > >> > > older
> > > > > >> > > > > > > email,
> > > > > >> > > > > > > > I
> > > > > >> > > > > > > > > >> >> am
> > > > > >> > > > > > > > > >> >>>>>>>>>> personally
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure
> > > functional
> > > > > >> > > interface,
> > > > > >> > > > > and
> > > > > >> > > > > > > add
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > WithRecordContext
> > > > with
> > > > > >> > method
> > > > > >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > > > > >> > > > > > > > > >> >>>>>>>>>> (or
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better
> > name)
> > > > --
> > > > > >> but
> > > > > >> > not
> > > > > >> > > > > > > > `close()`)
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > > > > >> > ValueMapperWithRecordContext
> > > > > >> > > > > > extends
> > > > > >> > > > > > > > > >> >>>>> ValueMapper,
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid
> > any
> > > > > >> > overload.
> > > > > >> > > Of
> > > > > >> > > > > > > course,
> > > > > >> > > > > > > > > we
> > > > > >> > > > > > > > > >> >>>> don't
> > > > > >> > > > > > > > > >> >>>>>>>>> get
> > > > > >> > > > > > > > > >> >>>>>>>>>> a
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface
> and
> > > also
> > > > > >> > > sacrifices
> > > > > >> > > > > > > > Lambdas.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit
> > > > > undecided
> > > > > >> > what
> > > > > >> > > > the
> > > > > >> > > > > > > > better
> > > > > >> > > > > > > > > >> >>>> option
> > > > > >> > > > > > > > > >> >>>>>>>>> might
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>> be.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other
> > think
> > > > > about
> > > > > >> > this
> > > > > >> > > > > trade
> > > > > >> > > > > > > > off.
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun
> > > Karimov
> > > > > >> wrote:<
> > >
> >
>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
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<V1, V2, VR, K> {
    VR apply(final V1 value1, final V2 value2, final K key, final RecordContext
recordContext);
}

My motivation is that for lambda expression in J8, users that would not
care about the key but only the context, or vice versa, is likely to write
it as (value1, value2, dummy, context) -> ... than putting the dummy at the
beginning of the parameter list. Generally speaking we'd like to make all
the "necessary" parameters prior to optional ones.


2. Some of the listed functions are not necessary since their pairing APIs
are being deprecated in 1.0 already:

<KR> KGroupedStream<KR, V> groupBy(final RichKeyValueMapper<? super K, ?
super V, KR> selector,
                                   final Serde<KR> keySerde,
                                   final Serde<V> valSerde);

<VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> table,
                                 final RichValueJoiner<? super K, ? super V,
? super VT, ? extends VR> joiner,
                                 final Serde<K> keySerde,
                                 final Serde<V> valSerde);



3. For a few functions where we are adding three APIs for a combo of both
mapper / joiner, or both initializer / aggregator, or adder / subtractor,
I'm wondering if we can just keep one that use "rich" functions for both;
so that we can have less overloads and let users who only want to access
one of them to just use dummy parameter declarations. For example:


<GK, GV, RV> KStream<K, RV> join(final GlobalKTable<GK, GV> globalKTable,
                                 final RichKeyValueMapper<? super K, ? super
 V, ? extends GK> keyValueMapper,
                                 final RichValueJoiner<? super K, ? super V,
? super GV, ? extends RV> joiner);

<VR> KTable<K, VR> aggregate(final RichInitializer<K, VR> initializer,
                             final RichAggregator<? super K, ? super V, VR>
aggregator,
                             final Materialized<K, VR, KeyValueStore<Bytes,
byte[]>> materialized);

Similarly for KGroupedTable, a bunch of aggregate() are deprecated so we do
not need to add its rich functions any more.


4. For TimeWindowedKStream, I'm wondering why we do not make its
Initializer also "rich" functions? I.e.

<VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
initializer,
                                       final RichAggregator<? super K, ?
super V, VR> aggregator);
<VR> KTable<Windowed<K>, VR> aggregate(final RichInitializer<VR, K>
initializer,
                                       final RichAggregator<? super K, ?
super V, VR> aggregator,
                                       final Materialized<K, VR,
WindowStore<Bytes, byte[]>> materialized);


5. We need to move "RecordContext" from o.a.k.processor.internals to
o.a.k.processor.

6. I'm not clear why we want to move `commit()` from ProcessorContext to
RecordContext? Conceptually I think it would better staying in the
ProcessorContext. Do you find this not doable in the internal
implementations?


Guozhang



On Fri, Sep 22, 2017 at 1:09 PM, Ted Yu <yu...@gmail.com> wrote:

>    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 <je...@gmail.com>
> wrote:
>
> > 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 to KIP.
> >
> >
> > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >                              final Aggregator<? super K, ? super V, VR>
> > > adder,
> > > The adder doesn't need to be RichAggregator ?
> >
> >
> >
> > - Exactly. However, there are 2 Aggregator-type arguments in the related
> > method. So, I had to overload all possible their Rich counterparts:
> >
> > // adder with non-rich, subtrctor is rich
> > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >                              final Aggregator<? super K, ? super V, VR>
> > adder,
> >                              final RichAggregator<? super K, ? super V,
> VR>
> > subtractor,
> >                              final Materialized<K, VR,
> KeyValueStore<Bytes,
> > byte[]>> materialized);
> >
> > // adder withrich, subtrctor is non-rich
> > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >                              final RichAggregator<? super K, ? super V,
> VR>
> > adder,
> >                              final Aggregator<? super K, ? super V, VR>
> > subtractor,
> >                              final Materialized<K, VR,
> KeyValueStore<Bytes,
> > byte[]>> materialized);
> >
> > // both adder and subtractor are rich
> > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >                              final RichAggregator<? super K, ? super V,
> VR>
> > adder,
> >                              final RichAggregator<? super K, ? super V,
> VR>
> > subtractor,
> >                              final Materialized<K, VR,
> KeyValueStore<Bytes,
> > byte[]>> materialized);
> >
> >
> > Can you explain a bit about the above implementation ?
> > >    void commit () {
> > >      throw new UnsupportedOperationException("commit() is not
> supported
> > in
> > > this context");
> > > Is the exception going to be replaced with real code in the PR ?
> >
> >
> >
> > - I added some comments both inside and outside the code snippets in KIP.
> > Specifically, for the code snippet above, we add *commit()* method to
> > *RecordContext* interface.
> > However, we want  *commit()* method to be used only for *RecordContext*
> > instances (at least for now), so we add UnsupportedOperationException in
> > all classes/interfaces that extend/implement *RecordContext.*
> > In general, 1) we make RecordContext publicly available within
> > ProcessorContext,  2) initialize its instance within all required
> > Processors and 3) pass it as an argument to the related Rich interfaces
> > inside Processors.
> >
> >
> >
> >
> > Cheers,
> > Jeyhun
> >
> > On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com> wrote:
> >
> > > 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.
> > >
> > > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> > >                              final Aggregator<? super K, ? super V, VR>
> > > adder,
> > >
> > > The adder doesn't need to be RichAggregator ?
> > >
> > >   public RecordContext recordContext() {
> > >     return this.recordContext();
> > >
> > > Can you explain a bit about the above implementation ?
> > >
> > >    void commit () {
> > >      throw new UnsupportedOperationException("commit() is not
> supported
> > in
> > > this context");
> > >
> > > Is the exception going to be replaced with real code in the PR ?
> > >
> > > Cheers
> > >
> > >
> > > On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <je...@gmail.com>
> > > wrote:
> > >
> > > > 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 <
> je.karimov@gmail.com>
> > > > wrote:
> > > >
> > > > > 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 <da...@gmail.com>
> > > 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 Jeyhun Karimov <je.karimov@gmail.com
> >
> > > > wrote:
> > > > >>
> > > > >> > 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 merge this KIP into the upcoming release, as KIP-159 is not
> > voted
> > > > yet
> > > > >> > (because we want both KIP-149 and KIP-159 to be as an "atomic"
> > > merge).
> > > > >> So
> > > > >> > I decided to wait until KIP-182 gets stable (there are some
> minor
> > > > >> updates
> > > > >> > AFAIK) and update the KIP accordingly. Please correct me if I am
> > > wrong
> > > > >> or I
> > > > >> > misunderstood.
> > > > >> >
> > > > >> > Cheers,
> > > > >> > Jeyhun
> > > > >> >
> > > > >> >
> > > > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <
> damian.guy@gmail.com>
> > > > >> wrote:
> > > > >> >
> > > > >> > > +1
> > > > >> > >
> > > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <
> wangguoz@gmail.com>
> > > > >> 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 would
> > still
> > > > >> need to
> > > > >> > > > update your proposal for people to comment and vote on.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > Guozhang
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <
> yuzhihong@gmail.com
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > +1
> > > > >> > > > >
> > > > >> > > > > One interface is cleaner.
> > > > >> > > > >
> > > > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > > bbejeck@gmail.com
> > > > >
> > > > >> > > wrote:
> > > > >> > > > >
> > > > >> > > > > > +1 for me on collapsing the RichXXXX and
> ValueXXXXWithKey
> > > > >> > interfaces
> > > > >> > > > > into 1
> > > > >> > > > > > interface.
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Bill
> > > > >> > > > > >
> > > > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > > >> > > je.karimov@gmail.com
> > > > >> > > > >
> > > > >> > > > > > 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 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
> > > > >> > > > > > this
> > > > >> > > > > > > would not be a problem.
> > > > >> > > > > > >
> > > > >> > > > > > > Our initial idea was similar to :
> > > > >> > > > > > >
> > > > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> > > implements
> > > > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > >> > > > > > > ......
> > > > >> > > > > > > }
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > So, we check the type of object, whether it is RichXXX
> > or
> > > > >> > > XXXWithKey
> > > > >> > > > > > inside
> > > > >> > > > > > > the called method and continue accordingly.
> > > > >> > > > > > >
> > > > >> > > > > > > If this is ok with the community, I would like to
> revert
> > > the
> > > > >> > > current
> > > > >> > > > > > design
> > > > >> > > > > > > to this again.
> > > > >> > > > > > >
> > > > >> > > > > > > Cheers,
> > > > >> > > > > > > Jeyhun
> > > > >> > > > > > >
> > > > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > > >> damian.guy@gmail.com
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hi Jeyhun,
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks for sending out the update. I guess i was
> > > thinking
> > > > >> more
> > > > >> > > > along
> > > > >> > > > > > the
> > > > >> > > > > > > > lines of option 2 where we collapse the RichXXXX and
> > > > >> > > > ValueXXXXWithKey
> > > > >> > > > > > 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?
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Damian
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > > >> > > je.karimov@gmail.com>
> > > > >> > > > > > > wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > 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 free to comment on this.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > [1]
> > > > >> > > > > > > > >
> > > > >> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > > >> > > > > > > action?pageId=73637757
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Cheers,
> > > > >> > > > > > > > > Jeyhun
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > > >> > > > > je.karimov@gmail.com
> > > > >> > > > > > >
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > 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 <
> > > > >> > > > > > > matthias@confluent.io>
> > > > >> > > > > > > > > > wrote:
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > >> 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
> > > > >> > > > > > > > > >> > for one-shot solution for overloads issue.
> > > > >> > > > > > > > > >> >
> > > > >> > > > > > > > > >> > Cheers,
> > > > >> > > > > > > > > >> > Jeyhun
> > > > >> > > > > > > > > >> >
> > > > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > > >> > > > > > damian.guy@gmail.com>
> > > > >> > > > > > > > > >> wrote:
> > > > >> > > > > > > > > >> >
> > > > >> > > > > > > > > >> >> 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
> > > > >> > > > > > > > > overrides.
> > > > >> > > > > > > > > >> >>
> > > > >> > > > > > > > > >> >> Thanks,
> > > > >> > > > > > > > > >> >> Damian
> > > > >> > > > > > > > > >> >>
> > > > >> > > > > > > > > >> >>
> > > > >> > > > > > > > > >> >>> About ProcessorContext vs RecordContext,
> you
> > > are
> > > > >> > right.
> > > > >> > > I
> > > > >> > > > > > think
> > > > >> > > > > > > I
> > > > >> > > > > > > > > >> need to
> > > > >> > > > > > > > > >> >>> implement a prototype to understand the
> full
> > > > >> picture
> > > > >> > as
> > > > >> > > > some
> > > > >> > > > > > > parts
> > > > >> > > > > > > > > of
> > > > >> > > > > > > > > >> the
> > > > >> > > > > > > > > >> >>> KIP might not be as straightforward as I
> > > thought.
> > > > >> > > > > > > > > >> >>>
> > > > >> > > > > > > > > >> >>>
> > > > >> > > > > > > > > >> >>> Cheers,
> > > > >> > > > > > > > > >> >>> Jeyhun
> > > > >> > > > > > > > > >> >>>
> > > > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy
> <
> > > > >> > > > > > > damian.guy@gmail.com>
> > > > >> > > > > > > > > >> wrote:
> > > > >> > > > > > > > > >> >>>
> > > > >> > > > > > > > > >> >>>> 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
> > > > >> > > > > lifetime
> > > > >> > > > > > of
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to
> > cast a
> > > > >> > > > > > ProcessorContext
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > a
> > > > >> > > > > > > > > >> >>>> RecordContext.
> > > > >> > > > > > > > > >> >>>> You mentioned above passing the
> > > > >> > > InternalProcessorContext
> > > > >> > > > to
> > > > >> > > > > > the
> > > > >> > > > > > > > > >> init()
> > > > >> > > > > > > > > >> >>>> calls. It is internal for a reason and i
> > think
> > > > it
> > > > >> > > should
> > > > >> > > > > > remain
> > > > >> > > > > > > > > that
> > > > >> > > > > > > > > >> >> way.
> > > > >> > > > > > > > > >> >>>> It might be better to move the
> > recordContext()
> > > > >> method
> > > > >> > > > from
> > > > >> > > > > > > > > >> >>>> InternalProcessorContext to
> > ProcessorContext.
> > > > >> > > > > > > > > >> >>>>
> > > > >> > > > > > > > > >> >>>> In the KIP you have an example showing:
> > > > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > > > processorContext);
> > > > >> > > > > > > > > >> >>>> But the interface is:
> > > > >> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> > > RecordContext
> > > > >> > > > > > recordContext);
> > > > >> > > > > > > > > >> >>>> }
> > > > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides as
> > above
> > > > this
> > > > >> > > > wouldn't
> > > > >> > > > > > > make
> > > > >> > > > > > > > > >> sense.
> > > > >> > > > > > > > > >> >>>>
> > > > >> > > > > > > > > >> >>>> Thanks,
> > > > >> > > > > > > > > >> >>>> Damian
> > > > >> > > > > > > > > >> >>>>
> > > > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun
> Karimov <
> > > > >> > > > > > > je.karimov@gmail.com
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > >> >> wrote:
> > > > >> > > > > > > > > >> >>>>
> > > > >> > > > > > > > > >> >>>>> 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.
> > > > >> Can't
> > > > >> > we
> > > > >> > > > pass
> > > > >> > > > > > > this
> > > > >> > > > > > > > > as a
> > > > >> > > > > > > > > >> >>>>> parameter to init() method of processors?
> > > Then
> > > > we
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > > able
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > >> >> get
> > > > >> > > > > > > > > >> >>>>> RecordContext easily with just a method
> > call.
> > > > >> > > > > > > > > >> >>>>>
> > > > >> > > > > > > > > >> >>>>>
> > > > >> > > > > > > > > >> >>>>> Cheers,
> > > > >> > > > > > > > > >> >>>>> Jeyhun
> > > > >> > > > > > > > > >> >>>>>
> > > > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias
> > J.
> > > > Sax
> > > > >> <
> > > > >> > > > > > > > > >> >>> matthias@confluent.io>
> > > > >> > > > > > > > > >> >>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>
> > > > >> > > > > > > > > >> >>>>>> 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
> > > > >> > > > > > > > > >> >>> will
> > > > >> > > > > > > > > >> >>>>>> be provided by the consecutive call to
> > > > aggregate
> > > > >> > > > anyways.
> > > > >> > > > > > > > > >> >>>>>>
> > > > >> > > > > > > > > >> >>>>>>
> > > > >> > > > > > > > > >> >>>>>> -Matthias
> > > > >> > > > > > > > > >> >>>>>>
> > > > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax
> wrote:
> > > > >> > > > > > > > > >> >>>>>>> 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
> > implement
> > > > >> both.
> > > > >> > > This
> > > > >> > > > > > allows
> > > > >> > > > > > > > us
> > > > >> > > > > > > > > >> >> to
> > > > >> > > > > > > > > >> >>>> cast
> > > > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit
> the
> > > > >> visible
> > > > >> > > > scope.
> > > > >> > > > > > > > > >> >>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>
> > > > >> > > > > > > > > >> >>>>>>> -Matthias
> > > > >> > > > > > > > > >> >>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>
> > > > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov
> wrote:
> > > > >> > > > > > > > > >> >>>>>>>> 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 address this issue as
> well
> > > or
> > > > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > > > >> > > > > > > > > >> >>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>> Cheers,
> > > > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > > > >> > > > > > > > > >> >>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> > Guozhang
> > > > >> Wang <
> > > > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > > >> > > > > > > > > >> >>>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > >> > > > > > > > > >> >>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > > > >> > > > > > > > > >> >>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM,
> Jeyhun
> > > > >> Karimov
> > > > >> > <
> > > > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > >> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907
> > [1]
> > > > can
> > > > >> > also
> > > > >> > > > be
> > > > >> > > > > > > > > >> >> integrated
> > > > >> > > > > > > > > >> >>>> into
> > > > >> > > > > > > > > >> >>>>>> the
> > > > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> > > > >> > RecordContext
> > > > >> > > > > > > > interface.
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>> [1]
> > > > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM
> > Matthias
> > > > J.
> > > > >> > Sax <
> > > > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> 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
> > > > information
> > > > >> > only.
> > > > >> > > > If
> > > > >> > > > > > you
> > > > >> > > > > > > > want
> > > > >> > > > > > > > > >> >> to
> > > > >> > > > > > > > > >> >>>> do
> > > > >> > > > > > > > > >> >>>>> a
> > > > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should
> > rather
> > > > use
> > > > >> > > > > > #transform().
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we
> would
> > > > need
> > > > >> to
> > > > >> > > > switch
> > > > >> > > > > > to
> > > > >> > > > > > > a
> > > > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many
> > more
> > > > >> > > overloads.
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a
> > > simple
> > > > >> > > > interface
> > > > >> > > > > > > with a
> > > > >> > > > > > > > > >> >>> single
> > > > >> > > > > > > > > >> >>>>>>>>> method
> > > > >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext
> > object.
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang
> > wrote:
> > > > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > > > >> > > <
> > > > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > >> > > > > > > > > >> >>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will
> not
> > > > >> mention
> > > > >> > > > them.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are
> spread
> > in
> > > > >> > KIP-149
> > > > >> > > > and
> > > > >> > > > > in
> > > > >> > > > > > > > this
> > > > >> > > > > > > > > >> >> KIP
> > > > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions
> came
> > > into
> > > > >> the
> > > > >> > > > stage
> > > > >> > > > > > with
> > > > >> > > > > > > > > >> >>> KIP-149,
> > > > >> > > > > > > > > >> >>>>> in
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we
> > > > >> extended
> > > > >> > > > KIP-149
> > > > >> > > > > > to
> > > > >> > > > > > > > > >> >> support
> > > > >> > > > > > > > > >> >>>>> Rich
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich
> > functions,
> > > > we
> > > > >> > > > provided
> > > > >> > > > > > init
> > > > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian
> > suggested
> > > > >> that
> > > > >> > we
> > > > >> > > > > should
> > > > >> > > > > > > not
> > > > >> > > > > > > > > >> >>>> provide
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a
> > > result,
> > > > >> we
> > > > >> > > > > separated
> > > > >> > > > > > > the
> > > > >> > > > > > > > > >> >> two
> > > > >> > > > > > > > > >> >>>>>>>>> problems
> > > > >> > > > > > > > > >> >>>>>>>>>>> into
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems
> they
> > > can
> > > > >> be
> > > > >> > > > solved
> > > > >> > > > > in
> > > > >> > > > > > > > > >> >>> parallel.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was
> :
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > > ValueMapperWithKey<K,
> > > > V,
> > > > >> > VR>
> > > > >> > > {
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V
> > > > value);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> RichValueMapper<K,
> > > V,
> > > > >> VR>
> > > > >> > > > extends
> > > > >> > > > > > > > > >> >>>> RichFunction{
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > > > >> recordContext);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > appConfigs();
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > >> > > > > appConfigsWithPrefix(String
> > > > >> > > > > > > > > >> >> prefix);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext
> > > > extends
> > > > >> > > > > > > RecordContext
> > > > >> > > > > > > > {
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> > > > >> > > RecordContext
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and
> "withoutKey"
> > > > >> > interfaces
> > > > >> > > > can
> > > > >> > > > > be
> > > > >> > > > > > > > > >> >>> converted
> > > > >> > > > > > > > > >> >>>> to
> > > > >> > > > > > > > > >> >>>>>>>>>> their
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty
> > init()
> > > > and
> > > > >> > > close()
> > > > >> > > > > > > > methods)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will
> > accept
> > > > >> Rich
> > > > >> > > > > > interfaces
> > > > >> > > > > > > in
> > > > >> > > > > > > > > >> >>> their
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related
> > > "withKey"
> > > > >> or
> > > > >> > > > > > > "withoutKey"
> > > > >> > > > > > > > > >> >>>>> interfaces
> > > > >> > > > > > > > > >> >>>>>>>>> to
> > > > >> > > > > > > > > >> >>>>>>>>>>> Rich
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the
> > topology
> > > > and
> > > > >> > > > > initialize
> > > > >> > > > > > > the
> > > > >> > > > > > > > > >> >>>> related
> > > > >> > > > > > > > > >> >>>>>>>>>>> processors
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded
> > > > methods
> > > > >> > for
> > > > >> > > > rich
> > > > >> > > > > > > > > >> >> functions
> > > > >> > > > > > > > > >> >>> as
> > > > >> > > > > > > > > >> >>>>>> Rich
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey
> > interfaces.
> > > > We
> > > > >> > will
> > > > >> > > > just
> > > > >> > > > > > > check
> > > > >> > > > > > > > > >> >> the
> > > > >> > > > > > > > > >> >>>>> object
> > > > >> > > > > > > > > >> >>>>>>>>>> type
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that
> the
> > > > above
> > > > >> > > > approach
> > > > >> > > > > > > does
> > > > >> > > > > > > > > not
> > > > >> > > > > > > > > >> >>>>> support
> > > > >> > > > > > > > > >> >>>>>>>>>>> lambdas
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one
> > method,
> > > > >> only
> > > > >> > > > > > > > > >> >>> init(RecordContext),
> > > > >> > > > > > > > > >> >>>>> as
> > > > >> > > > > > > > > >> >>>>>>>>>> part
> > > > >> > > > > > > > > >> >>>>>>>>>>> of
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> > > > Personally I
> > > > >> > > think
> > > > >> > > > > Rich
> > > > >> > > > > > > > > >> >>> interfaces
> > > > >> > > > > > > > > >> >>>>> are
> > > > >> > > > > > > > > >> >>>>>>>>> by
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we
> > should
> > > > not
> > > > >> > care
> > > > >> > > > much
> > > > >> > > > > > > about
> > > > >> > > > > > > > > >> >> it.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's
> discussion,
> > > an
> > > > >> > > > alternative
> > > > >> > > > > > we
> > > > >> > > > > > > > > >> >>>> considered
> > > > >> > > > > > > > > >> >>>>>> was
> > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as
> method
> > > > >> > parameter.
> > > > >> > > > > This
> > > > >> > > > > > > > might
> > > > >> > > > > > > > > >> >>> even
> > > > >> > > > > > > > > >> >>>>>>>>> allow
> > > > >> > > > > > > > > >> >>>>>>>>>> to
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the
> > > name
> > > > >> > > > > RichFunction
> > > > >> > > > > > as
> > > > >> > > > > > > > we
> > > > >> > > > > > > > > >> >>>>> preserve
> > > > >> > > > > > > > > >> >>>>>>>>> the
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and
> > > `close()`
> > > > we
> > > > >> > > > > basically
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory
> > > state
> > > > >> for a
> > > > >> > > > > > function.
> > > > >> > > > > > > > > >> >> Thus,
> > > > >> > > > > > > > > >> >>> we
> > > > >> > > > > > > > > >> >>>>>>>>> cannot
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> > > > >> RichValueMapper
> > > > >> > > (etc)
> > > > >> > > > > > over
> > > > >> > > > > > > > > >> >>> multiple
> > > > >> > > > > > > > > >> >>>>>> tasks
> > > > >> > > > > > > > > >> >>>>>>>>>> and
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern
> > > > similar
> > > > >> to
> > > > >> > > > > > > > #transform().
> > > > >> > > > > > > > > >> >> And
> > > > >> > > > > > > > > >> >>>>> this
> > > > >> > > > > > > > > >> >>>>>>>>>> would
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > >> > > > > > > > (Rich)ValueMapperSupplier
> > > > >> > > > > > > > > >> >>> would
> > > > >> > > > > > > > > >> >>>>> not
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus
> > we
> > > > >> would
> > > > >> > > need
> > > > >> > > > > many
> > > > >> > > > > > > new
> > > > >> > > > > > > > > >> >>>> overload
> > > > >> > > > > > > > > >> >>>>>>>>> for
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy
> > paste
> > > > from
> > > > >> > > > > Matthias's
> > > > >> > > > > > > > > email)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM
> > > Matthias
> > > > >> J.
> > > > >> > > Sax <
> > > > >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > >> > > > > > > > > >> >>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > > > >> > > > 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:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V,
> VR>
> > {
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> > > > >> > RecordContext
> > > > >> > > > > > > context);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > >> > mapValues(ValueMapper<?
> > > > >> > > > > super
> > > > >> > > > > > > V, ?
> > > > >> > > > > > > > > >> >>>> extends
> > > > >> > > > > > > > > >> >>>>>> VR>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > >> > > > > > > > mapValueswithContext(RichValueMapper
> > > > >> > > > > > > > > >> >> <?
> > > > >> > > > > > > > > >> >>>>> super
> > > > >> > > > > > > > > >> >>>>>>>>>> V, ?
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ------------------------------
> -
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will
> > > introduces
> > > > >> more
> > > > >> > > > > > > overloads;
> > > > >> > > > > > > > > >> >> but I
> > > > >> > > > > > > > > >> >>>>> think
> > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly
> introduced
> > > by
> > > > 1)
> > > > >> > > serde
> > > > >> > > > > > > > overrides
> > > > >> > > > > > > > > >> >> and
> > > > >> > > > > > > > > >> >>>> 2)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides,
> > both
> > > > of
> > > > >> > which
> > > > >> > > > can
> > > > >> > > > > > be
> > > > >> > > > > > > > > >> >> reduced
> > > > >> > > > > > > > > >> >>>> in
> > > > >> > > > > > > > > >> >>>>>> the
> > > > >> > > > > > > > > >> >>>>>>>>>>> near
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this
> > overloading
> > > > is
> > > > >> > still
> > > > >> > > > > > > > worthwhile,
> > > > >> > > > > > > > > >> >> as
> > > > >> > > > > > > > > >> >>>> it
> > > > >> > > > > > > > > >> >>>>>> has
> > > > >> > > > > > > > > >> >>>>>>>>>> the
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda
> > expressions.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not
> need
> > > to
> > > > >> > > "convert"
> > > > >> > > > > > from
> > > > >> > > > > > > > > >> >>> non-rich
> > > > >> > > > > > > > > >> >>>>>>>>>> functions
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> to
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already
> > > been
> > > > >> > > discussed
> > > > >> > > > > > and I
> > > > >> > > > > > > > may
> > > > >> > > > > > > > > >> >>> have
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>> in
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18
> PM,
> > > > >> Matthias
> > > > >> > J.
> > > > >> > > > > Sax <
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to
> > > > converge
> > > > >> to
> > > > >> > an
> > > > >> > > > > > overall
> > > > >> > > > > > > > > >> >>>> consisted
> > > > >> > > > > > > > > >> >>>>>>>>> API.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads.
> > Yes,
> > > > we
> > > > >> > might
> > > > >> > > > get
> > > > >> > > > > > > more
> > > > >> > > > > > > > > >> >>>> overload.
> > > > >> > > > > > > > > >> >>>>>> It
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> might
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a
> > > single
> > > > >> > > > > > > xxxWithContext()
> > > > >> > > > > > > > > >> >>>> overload
> > > > >> > > > > > > > > >> >>>>>>>>> that
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> will
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> > > > Otherwise,
> > > > >> if
> > > > >> > > > might
> > > > >> > > > > > get
> > > > >> > > > > > > > too
> > > > >> > > > > > > > > >> >>> messy
> > > > >> > > > > > > > > >> >>>>>>>>> having
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper,
> ValueMapperWithKey,
> > > > >> > > > > > > > > ValueMapperWithContext,
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also
> have
> > > the
> > > > >> > > "builder
> > > > >> > > > > > > pattern"
> > > > >> > > > > > > > > >> >> idea
> > > > >> > > > > > > > > >> >>>> as
> > > > >> > > > > > > > > >> >>>>> an
> > > > >> > > > > > > > > >> >>>>>>>>>> API
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate
> > the
> > > > >> > overload
> > > > >> > > > > > > problem.
> > > > >> > > > > > > > > Not
> > > > >> > > > > > > > > >> >>> for
> > > > >> > > > > > > > > >> >>>>>>>>> simple
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc
> > but
> > > > for
> > > > >> > joins
> > > > >> > > > and
> > > > >> > > > > > > > > >> >>>> aggregations.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I
> > mentioned
> > > in
> > > > >> an
> > > > >> > > older
> > > > >> > > > > > > email,
> > > > >> > > > > > > > I
> > > > >> > > > > > > > > >> >> am
> > > > >> > > > > > > > > >> >>>>>>>>>> personally
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure
> > functional
> > > > >> > > interface,
> > > > >> > > > > and
> > > > >> > > > > > > add
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> WithRecordContext
> > > with
> > > > >> > method
> > > > >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > > > >> > > > > > > > > >> >>>>>>>>>> (or
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better
> name)
> > > --
> > > > >> but
> > > > >> > not
> > > > >> > > > > > > > `close()`)
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > > > >> > ValueMapperWithRecordContext
> > > > >> > > > > > extends
> > > > >> > > > > > > > > >> >>>>> ValueMapper,
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid
> any
> > > > >> > overload.
> > > > >> > > Of
> > > > >> > > > > > > course,
> > > > >> > > > > > > > > we
> > > > >> > > > > > > > > >> >>>> don't
> > > > >> > > > > > > > > >> >>>>>>>>> get
> > > > >> > > > > > > > > >> >>>>>>>>>> a
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and
> > also
> > > > >> > > sacrifices
> > > > >> > > > > > > > Lambdas.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit
> > > > undecided
> > > > >> > what
> > > > >> > > > the
> > > > >> > > > > > > > better
> > > > >> > > > > > > > > >> >>>> option
> > > > >> > > > > > > > > >> >>>>>>>>> might
> > > > >> > > > > > > > > >> >>>>>>>>>>>>> be.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other
> think
> > > > about
> > > > >> > this
> > > > >> > > > > trade
> > > > >> > > > > > > > off.
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun
> > Karimov
> > > > >> wrote:<
> >
>



-- 
-- Guozhang

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

Posted by Ted Yu <yu...@gmail.com>.
   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 <je...@gmail.com>
wrote:

> 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 to KIP.
>
>
> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >                              final Aggregator<? super K, ? super V, VR>
> > adder,
> > The adder doesn't need to be RichAggregator ?
>
>
>
> - Exactly. However, there are 2 Aggregator-type arguments in the related
> method. So, I had to overload all possible their Rich counterparts:
>
> // adder with non-rich, subtrctor is rich
> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>                              final Aggregator<? super K, ? super V, VR>
> adder,
>                              final RichAggregator<? super K, ? super V, VR>
> subtractor,
>                              final Materialized<K, VR, KeyValueStore<Bytes,
> byte[]>> materialized);
>
> // adder withrich, subtrctor is non-rich
> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>                              final RichAggregator<? super K, ? super V, VR>
> adder,
>                              final Aggregator<? super K, ? super V, VR>
> subtractor,
>                              final Materialized<K, VR, KeyValueStore<Bytes,
> byte[]>> materialized);
>
> // both adder and subtractor are rich
> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>                              final RichAggregator<? super K, ? super V, VR>
> adder,
>                              final RichAggregator<? super K, ? super V, VR>
> subtractor,
>                              final Materialized<K, VR, KeyValueStore<Bytes,
> byte[]>> materialized);
>
>
> Can you explain a bit about the above implementation ?
> >    void commit () {
> >      throw new UnsupportedOperationException("commit() is not supported
> in
> > this context");
> > Is the exception going to be replaced with real code in the PR ?
>
>
>
> - I added some comments both inside and outside the code snippets in KIP.
> Specifically, for the code snippet above, we add *commit()* method to
> *RecordContext* interface.
> However, we want  *commit()* method to be used only for *RecordContext*
> instances (at least for now), so we add UnsupportedOperationException in
> all classes/interfaces that extend/implement *RecordContext.*
> In general, 1) we make RecordContext publicly available within
> ProcessorContext,  2) initialize its instance within all required
> Processors and 3) pass it as an argument to the related Rich interfaces
> inside Processors.
>
>
>
>
> Cheers,
> Jeyhun
>
> On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com> wrote:
>
> > 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.
> >
> > <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
> >                              final Aggregator<? super K, ? super V, VR>
> > adder,
> >
> > The adder doesn't need to be RichAggregator ?
> >
> >   public RecordContext recordContext() {
> >     return this.recordContext();
> >
> > Can you explain a bit about the above implementation ?
> >
> >    void commit () {
> >      throw new UnsupportedOperationException("commit() is not supported
> in
> > this context");
> >
> > Is the exception going to be replaced with real code in the PR ?
> >
> > Cheers
> >
> >
> > On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <je...@gmail.com>
> > wrote:
> >
> > > 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 <je...@gmail.com>
> > > wrote:
> > >
> > > > 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 <da...@gmail.com>
> > 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 Jeyhun Karimov <je...@gmail.com>
> > > wrote:
> > > >>
> > > >> > 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 merge this KIP into the upcoming release, as KIP-159 is not
> voted
> > > yet
> > > >> > (because we want both KIP-149 and KIP-159 to be as an "atomic"
> > merge).
> > > >> So
> > > >> > I decided to wait until KIP-182 gets stable (there are some minor
> > > >> updates
> > > >> > AFAIK) and update the KIP accordingly. Please correct me if I am
> > wrong
> > > >> or I
> > > >> > misunderstood.
> > > >> >
> > > >> > Cheers,
> > > >> > Jeyhun
> > > >> >
> > > >> >
> > > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> > > +1
> > > >> > >
> > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com>
> > > >> 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 would
> still
> > > >> need to
> > > >> > > > update your proposal for people to comment and vote on.
> > > >> > > >
> > > >> > > >
> > > >> > > > Guozhang
> > > >> > > >
> > > >> > > >
> > > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yuzhihong@gmail.com
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > +1
> > > >> > > > >
> > > >> > > > > One interface is cleaner.
> > > >> > > > >
> > > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> > bbejeck@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
> > > >> > interfaces
> > > >> > > > > into 1
> > > >> > > > > > interface.
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Bill
> > > >> > > > > >
> > > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > >> > > je.karimov@gmail.com
> > > >> > > > >
> > > >> > > > > > 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 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
> > > >> > > > > > this
> > > >> > > > > > > would not be a problem.
> > > >> > > > > > >
> > > >> > > > > > > Our initial idea was similar to :
> > > >> > > > > > >
> > > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> > implements
> > > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > >> > > > > > > ......
> > > >> > > > > > > }
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > So, we check the type of object, whether it is RichXXX
> or
> > > >> > > XXXWithKey
> > > >> > > > > > inside
> > > >> > > > > > > the called method and continue accordingly.
> > > >> > > > > > >
> > > >> > > > > > > If this is ok with the community, I would like to revert
> > the
> > > >> > > current
> > > >> > > > > > design
> > > >> > > > > > > to this again.
> > > >> > > > > > >
> > > >> > > > > > > Cheers,
> > > >> > > > > > > Jeyhun
> > > >> > > > > > >
> > > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > > >> damian.guy@gmail.com
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hi Jeyhun,
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks for sending out the update. I guess i was
> > thinking
> > > >> more
> > > >> > > > along
> > > >> > > > > > the
> > > >> > > > > > > > lines of option 2 where we collapse the RichXXXX and
> > > >> > > > ValueXXXXWithKey
> > > >> > > > > > 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?
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Damian
> > > >> > > > > > > >
> > > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > >> > > je.karimov@gmail.com>
> > > >> > > > > > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > 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 free to comment on this.
> > > >> > > > > > > > >
> > > >> > > > > > > > > [1]
> > > >> > > > > > > > >
> > > >> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > >> > > > > > > action?pageId=73637757
> > > >> > > > > > > > >
> > > >> > > > > > > > > Cheers,
> > > >> > > > > > > > > Jeyhun
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > >> > > > > je.karimov@gmail.com
> > > >> > > > > > >
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > > > 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 <
> > > >> > > > > > > matthias@confluent.io>
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > >> 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
> > > >> > > > > > > > > >> > for one-shot solution for overloads issue.
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > Cheers,
> > > >> > > > > > > > > >> > Jeyhun
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > >> > > > > > damian.guy@gmail.com>
> > > >> > > > > > > > > >> wrote:
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> >> 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
> > > >> > > > > > > > > overrides.
> > > >> > > > > > > > > >> >>
> > > >> > > > > > > > > >> >> Thanks,
> > > >> > > > > > > > > >> >> Damian
> > > >> > > > > > > > > >> >>
> > > >> > > > > > > > > >> >>
> > > >> > > > > > > > > >> >>> About ProcessorContext vs RecordContext, you
> > are
> > > >> > right.
> > > >> > > I
> > > >> > > > > > think
> > > >> > > > > > > I
> > > >> > > > > > > > > >> need to
> > > >> > > > > > > > > >> >>> implement a prototype to understand the full
> > > >> picture
> > > >> > as
> > > >> > > > some
> > > >> > > > > > > parts
> > > >> > > > > > > > > of
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> >>> KIP might not be as straightforward as I
> > thought.
> > > >> > > > > > > > > >> >>>
> > > >> > > > > > > > > >> >>>
> > > >> > > > > > > > > >> >>> Cheers,
> > > >> > > > > > > > > >> >>> Jeyhun
> > > >> > > > > > > > > >> >>>
> > > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > >> > > > > > > damian.guy@gmail.com>
> > > >> > > > > > > > > >> wrote:
> > > >> > > > > > > > > >> >>>
> > > >> > > > > > > > > >> >>>> 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
> > > >> > > > > lifetime
> > > >> > > > > > of
> > > >> > > > > > > > the
> > > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to
> cast a
> > > >> > > > > > ProcessorContext
> > > >> > > > > > > > to
> > > >> > > > > > > > > a
> > > >> > > > > > > > > >> >>>> RecordContext.
> > > >> > > > > > > > > >> >>>> You mentioned above passing the
> > > >> > > InternalProcessorContext
> > > >> > > > to
> > > >> > > > > > the
> > > >> > > > > > > > > >> init()
> > > >> > > > > > > > > >> >>>> calls. It is internal for a reason and i
> think
> > > it
> > > >> > > should
> > > >> > > > > > remain
> > > >> > > > > > > > > that
> > > >> > > > > > > > > >> >> way.
> > > >> > > > > > > > > >> >>>> It might be better to move the
> recordContext()
> > > >> method
> > > >> > > > from
> > > >> > > > > > > > > >> >>>> InternalProcessorContext to
> ProcessorContext.
> > > >> > > > > > > > > >> >>>>
> > > >> > > > > > > > > >> >>>> In the KIP you have an example showing:
> > > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > > processorContext);
> > > >> > > > > > > > > >> >>>> But the interface is:
> > > >> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> > RecordContext
> > > >> > > > > > recordContext);
> > > >> > > > > > > > > >> >>>> }
> > > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides as
> above
> > > this
> > > >> > > > wouldn't
> > > >> > > > > > > make
> > > >> > > > > > > > > >> sense.
> > > >> > > > > > > > > >> >>>>
> > > >> > > > > > > > > >> >>>> Thanks,
> > > >> > > > > > > > > >> >>>> Damian
> > > >> > > > > > > > > >> >>>>
> > > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > >> > > > > > > je.karimov@gmail.com
> > > >> > > > > > > > >
> > > >> > > > > > > > > >> >> wrote:
> > > >> > > > > > > > > >> >>>>
> > > >> > > > > > > > > >> >>>>> 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.
> > > >> Can't
> > > >> > we
> > > >> > > > pass
> > > >> > > > > > > this
> > > >> > > > > > > > > as a
> > > >> > > > > > > > > >> >>>>> parameter to init() method of processors?
> > Then
> > > we
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > > able
> > > >> > > > > > > > to
> > > >> > > > > > > > > >> >> get
> > > >> > > > > > > > > >> >>>>> RecordContext easily with just a method
> call.
> > > >> > > > > > > > > >> >>>>>
> > > >> > > > > > > > > >> >>>>>
> > > >> > > > > > > > > >> >>>>> Cheers,
> > > >> > > > > > > > > >> >>>>> Jeyhun
> > > >> > > > > > > > > >> >>>>>
> > > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias
> J.
> > > Sax
> > > >> <
> > > >> > > > > > > > > >> >>> matthias@confluent.io>
> > > >> > > > > > > > > >> >>>>> wrote:
> > > >> > > > > > > > > >> >>>>>
> > > >> > > > > > > > > >> >>>>>> 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
> > > >> > > > > > > > > >> >>> will
> > > >> > > > > > > > > >> >>>>>> be provided by the consecutive call to
> > > aggregate
> > > >> > > > anyways.
> > > >> > > > > > > > > >> >>>>>>
> > > >> > > > > > > > > >> >>>>>>
> > > >> > > > > > > > > >> >>>>>> -Matthias
> > > >> > > > > > > > > >> >>>>>>
> > > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > >> > > > > > > > > >> >>>>>>> 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
> implement
> > > >> both.
> > > >> > > This
> > > >> > > > > > allows
> > > >> > > > > > > > us
> > > >> > > > > > > > > >> >> to
> > > >> > > > > > > > > >> >>>> cast
> > > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the
> > > >> visible
> > > >> > > > scope.
> > > >> > > > > > > > > >> >>>>>>>
> > > >> > > > > > > > > >> >>>>>>>
> > > >> > > > > > > > > >> >>>>>>> -Matthias
> > > >> > > > > > > > > >> >>>>>>>
> > > >> > > > > > > > > >> >>>>>>>
> > > >> > > > > > > > > >> >>>>>>>
> > > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > >> > > > > > > > > >> >>>>>>>> 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 address this issue as well
> > or
> > > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > > >> > > > > > > > > >> >>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>> Cheers,
> > > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > > >> > > > > > > > > >> >>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM
> Guozhang
> > > >> Wang <
> > > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > >> > > > > > > > > >> >>>>>> wrote:
> > > >> > > > > > > > > >> >>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > >> > > > > > > > > >> >>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > > >> > > > > > > > > >> >>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun
> > > >> Karimov
> > > >> > <
> > > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > >> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907
> [1]
> > > can
> > > >> > also
> > > >> > > > be
> > > >> > > > > > > > > >> >> integrated
> > > >> > > > > > > > > >> >>>> into
> > > >> > > > > > > > > >> >>>>>> the
> > > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> > > >> > RecordContext
> > > >> > > > > > > > interface.
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>> [1]
> > > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM
> Matthias
> > > J.
> > > >> > Sax <
> > > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> 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
> > > information
> > > >> > only.
> > > >> > > > If
> > > >> > > > > > you
> > > >> > > > > > > > want
> > > >> > > > > > > > > >> >> to
> > > >> > > > > > > > > >> >>>> do
> > > >> > > > > > > > > >> >>>>> a
> > > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should
> rather
> > > use
> > > >> > > > > > #transform().
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would
> > > need
> > > >> to
> > > >> > > > switch
> > > >> > > > > > to
> > > >> > > > > > > a
> > > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many
> more
> > > >> > > overloads.
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a
> > simple
> > > >> > > > interface
> > > >> > > > > > > with a
> > > >> > > > > > > > > >> >>> single
> > > >> > > > > > > > > >> >>>>>>>>> method
> > > >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext
> object.
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang
> wrote:
> > > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > > >> > > <
> > > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > >> > > > > > > > > >> >>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > >> > > > > > > > > >> >>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will not
> > > >> mention
> > > >> > > > them.
> > > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread
> in
> > > >> > KIP-149
> > > >> > > > and
> > > >> > > > > in
> > > >> > > > > > > > this
> > > >> > > > > > > > > >> >> KIP
> > > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came
> > into
> > > >> the
> > > >> > > > stage
> > > >> > > > > > with
> > > >> > > > > > > > > >> >>> KIP-149,
> > > >> > > > > > > > > >> >>>>> in
> > > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we
> > > >> extended
> > > >> > > > KIP-149
> > > >> > > > > > to
> > > >> > > > > > > > > >> >> support
> > > >> > > > > > > > > >> >>>>> Rich
> > > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich
> functions,
> > > we
> > > >> > > > provided
> > > >> > > > > > init
> > > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian
> suggested
> > > >> that
> > > >> > we
> > > >> > > > > should
> > > >> > > > > > > not
> > > >> > > > > > > > > >> >>>> provide
> > > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a
> > result,
> > > >> we
> > > >> > > > > separated
> > > >> > > > > > > the
> > > >> > > > > > > > > >> >> two
> > > >> > > > > > > > > >> >>>>>>>>> problems
> > > >> > > > > > > > > >> >>>>>>>>>>> into
> > > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they
> > can
> > > >> be
> > > >> > > > solved
> > > >> > > > > in
> > > >> > > > > > > > > >> >>> parallel.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> > ValueMapperWithKey<K,
> > > V,
> > > >> > VR>
> > > >> > > {
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V
> > > value);
> > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K,
> > V,
> > > >> VR>
> > > >> > > > extends
> > > >> > > > > > > > > >> >>>> RichFunction{
> > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > > >> recordContext);
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> appConfigs();
> > > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > >> > > > > appConfigsWithPrefix(String
> > > >> > > > > > > > > >> >> prefix);
> > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext
> > > extends
> > > >> > > > > > > RecordContext
> > > >> > > > > > > > {
> > > >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> > > >> > > RecordContext
> > > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
> > > >> > interfaces
> > > >> > > > can
> > > >> > > > > be
> > > >> > > > > > > > > >> >>> converted
> > > >> > > > > > > > > >> >>>> to
> > > >> > > > > > > > > >> >>>>>>>>>> their
> > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty
> init()
> > > and
> > > >> > > close()
> > > >> > > > > > > > methods)
> > > >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will
> accept
> > > >> Rich
> > > >> > > > > > interfaces
> > > >> > > > > > > in
> > > >> > > > > > > > > >> >>> their
> > > >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related
> > "withKey"
> > > >> or
> > > >> > > > > > > "withoutKey"
> > > >> > > > > > > > > >> >>>>> interfaces
> > > >> > > > > > > > > >> >>>>>>>>> to
> > > >> > > > > > > > > >> >>>>>>>>>>> Rich
> > > >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the
> topology
> > > and
> > > >> > > > > initialize
> > > >> > > > > > > the
> > > >> > > > > > > > > >> >>>> related
> > > >> > > > > > > > > >> >>>>>>>>>>> processors
> > > >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded
> > > methods
> > > >> > for
> > > >> > > > rich
> > > >> > > > > > > > > >> >> functions
> > > >> > > > > > > > > >> >>> as
> > > >> > > > > > > > > >> >>>>>> Rich
> > > >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey
> interfaces.
> > > We
> > > >> > will
> > > >> > > > just
> > > >> > > > > > > check
> > > >> > > > > > > > > >> >> the
> > > >> > > > > > > > > >> >>>>> object
> > > >> > > > > > > > > >> >>>>>>>>>> type
> > > >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the
> > > above
> > > >> > > > approach
> > > >> > > > > > > does
> > > >> > > > > > > > > not
> > > >> > > > > > > > > >> >>>>> support
> > > >> > > > > > > > > >> >>>>>>>>>>> lambdas
> > > >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one
> method,
> > > >> only
> > > >> > > > > > > > > >> >>> init(RecordContext),
> > > >> > > > > > > > > >> >>>>> as
> > > >> > > > > > > > > >> >>>>>>>>>> part
> > > >> > > > > > > > > >> >>>>>>>>>>> of
> > > >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> > > Personally I
> > > >> > > think
> > > >> > > > > Rich
> > > >> > > > > > > > > >> >>> interfaces
> > > >> > > > > > > > > >> >>>>> are
> > > >> > > > > > > > > >> >>>>>>>>> by
> > > >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we
> should
> > > not
> > > >> > care
> > > >> > > > much
> > > >> > > > > > > about
> > > >> > > > > > > > > >> >> it.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion,
> > an
> > > >> > > > alternative
> > > >> > > > > > we
> > > >> > > > > > > > > >> >>>> considered
> > > >> > > > > > > > > >> >>>>>> was
> > > >> > > > > > > > > >> >>>>>>>>>> to
> > > >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
> > > >> > parameter.
> > > >> > > > > This
> > > >> > > > > > > > might
> > > >> > > > > > > > > >> >>> even
> > > >> > > > > > > > > >> >>>>>>>>> allow
> > > >> > > > > > > > > >> >>>>>>>>>> to
> > > >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the
> > name
> > > >> > > > > RichFunction
> > > >> > > > > > as
> > > >> > > > > > > > we
> > > >> > > > > > > > > >> >>>>> preserve
> > > >> > > > > > > > > >> >>>>>>>>> the
> > > >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and
> > `close()`
> > > we
> > > >> > > > > basically
> > > >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory
> > state
> > > >> for a
> > > >> > > > > > function.
> > > >> > > > > > > > > >> >> Thus,
> > > >> > > > > > > > > >> >>> we
> > > >> > > > > > > > > >> >>>>>>>>> cannot
> > > >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> > > >> RichValueMapper
> > > >> > > (etc)
> > > >> > > > > > over
> > > >> > > > > > > > > >> >>> multiple
> > > >> > > > > > > > > >> >>>>>> tasks
> > > >> > > > > > > > > >> >>>>>>>>>> and
> > > >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern
> > > similar
> > > >> to
> > > >> > > > > > > > #transform().
> > > >> > > > > > > > > >> >> And
> > > >> > > > > > > > > >> >>>>> this
> > > >> > > > > > > > > >> >>>>>>>>>> would
> > > >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > >> > > > > > > > (Rich)ValueMapperSupplier
> > > >> > > > > > > > > >> >>> would
> > > >> > > > > > > > > >> >>>>> not
> > > >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus
> we
> > > >> would
> > > >> > > need
> > > >> > > > > many
> > > >> > > > > > > new
> > > >> > > > > > > > > >> >>>> overload
> > > >> > > > > > > > > >> >>>>>>>>> for
> > > >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy
> paste
> > > from
> > > >> > > > > Matthias's
> > > >> > > > > > > > > email)
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM
> > Matthias
> > > >> J.
> > > >> > > Sax <
> > > >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > >> > > > > > > > > >> >>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > >> > > > > > > > > >> >>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > > >> > > > 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:
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR>
> {
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> > > >> > RecordContext
> > > >> > > > > > > context);
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > >> > mapValues(ValueMapper<?
> > > >> > > > > super
> > > >> > > > > > > V, ?
> > > >> > > > > > > > > >> >>>> extends
> > > >> > > > > > > > > >> >>>>>> VR>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > >> > > > > > > > mapValueswithContext(RichValueMapper
> > > >> > > > > > > > > >> >> <?
> > > >> > > > > > > > > >> >>>>> super
> > > >> > > > > > > > > >> >>>>>>>>>> V, ?
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will
> > introduces
> > > >> more
> > > >> > > > > > > overloads;
> > > >> > > > > > > > > >> >> but I
> > > >> > > > > > > > > >> >>>>> think
> > > >> > > > > > > > > >> >>>>>>>>>> the
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced
> > by
> > > 1)
> > > >> > > serde
> > > >> > > > > > > > overrides
> > > >> > > > > > > > > >> >> and
> > > >> > > > > > > > > >> >>>> 2)
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides,
> both
> > > of
> > > >> > which
> > > >> > > > can
> > > >> > > > > > be
> > > >> > > > > > > > > >> >> reduced
> > > >> > > > > > > > > >> >>>> in
> > > >> > > > > > > > > >> >>>>>> the
> > > >> > > > > > > > > >> >>>>>>>>>>> near
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this
> overloading
> > > is
> > > >> > still
> > > >> > > > > > > > worthwhile,
> > > >> > > > > > > > > >> >> as
> > > >> > > > > > > > > >> >>>> it
> > > >> > > > > > > > > >> >>>>>> has
> > > >> > > > > > > > > >> >>>>>>>>>> the
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda
> expressions.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need
> > to
> > > >> > > "convert"
> > > >> > > > > > from
> > > >> > > > > > > > > >> >>> non-rich
> > > >> > > > > > > > > >> >>>>>>>>>> functions
> > > >> > > > > > > > > >> >>>>>>>>>>>>> to
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already
> > been
> > > >> > > discussed
> > > >> > > > > > and I
> > > >> > > > > > > > may
> > > >> > > > > > > > > >> >>> have
> > > >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > >> > > > > > > > > >> >>>>>>>>>>>>>> in
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM,
> > > >> Matthias
> > > >> > J.
> > > >> > > > > Sax <
> > > >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to
> > > converge
> > > >> to
> > > >> > an
> > > >> > > > > > overall
> > > >> > > > > > > > > >> >>>> consisted
> > > >> > > > > > > > > >> >>>>>>>>> API.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads.
> Yes,
> > > we
> > > >> > might
> > > >> > > > get
> > > >> > > > > > > more
> > > >> > > > > > > > > >> >>>> overload.
> > > >> > > > > > > > > >> >>>>>> It
> > > >> > > > > > > > > >> >>>>>>>>>>>>> might
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a
> > single
> > > >> > > > > > > xxxWithContext()
> > > >> > > > > > > > > >> >>>> overload
> > > >> > > > > > > > > >> >>>>>>>>> that
> > > >> > > > > > > > > >> >>>>>>>>>>>>> will
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> > > Otherwise,
> > > >> if
> > > >> > > > might
> > > >> > > > > > get
> > > >> > > > > > > > too
> > > >> > > > > > > > > >> >>> messy
> > > >> > > > > > > > > >> >>>>>>>>> having
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > >> > > > > > > > > ValueMapperWithContext,
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have
> > the
> > > >> > > "builder
> > > >> > > > > > > pattern"
> > > >> > > > > > > > > >> >> idea
> > > >> > > > > > > > > >> >>>> as
> > > >> > > > > > > > > >> >>>>> an
> > > >> > > > > > > > > >> >>>>>>>>>> API
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate
> the
> > > >> > overload
> > > >> > > > > > > problem.
> > > >> > > > > > > > > Not
> > > >> > > > > > > > > >> >>> for
> > > >> > > > > > > > > >> >>>>>>>>> simple
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc
> but
> > > for
> > > >> > joins
> > > >> > > > and
> > > >> > > > > > > > > >> >>>> aggregations.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I
> mentioned
> > in
> > > >> an
> > > >> > > older
> > > >> > > > > > > email,
> > > >> > > > > > > > I
> > > >> > > > > > > > > >> >> am
> > > >> > > > > > > > > >> >>>>>>>>>> personally
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure
> functional
> > > >> > > interface,
> > > >> > > > > and
> > > >> > > > > > > add
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext
> > with
> > > >> > method
> > > >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > > >> > > > > > > > > >> >>>>>>>>>> (or
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name)
> > --
> > > >> but
> > > >> > not
> > > >> > > > > > > > `close()`)
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > > >> > ValueMapperWithRecordContext
> > > >> > > > > > extends
> > > >> > > > > > > > > >> >>>>> ValueMapper,
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
> > > >> > overload.
> > > >> > > Of
> > > >> > > > > > > course,
> > > >> > > > > > > > > we
> > > >> > > > > > > > > >> >>>> don't
> > > >> > > > > > > > > >> >>>>>>>>> get
> > > >> > > > > > > > > >> >>>>>>>>>> a
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and
> also
> > > >> > > sacrifices
> > > >> > > > > > > > Lambdas.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit
> > > undecided
> > > >> > what
> > > >> > > > the
> > > >> > > > > > > > better
> > > >> > > > > > > > > >> >>>> option
> > > >> > > > > > > > > >> >>>>>>>>> might
> > > >> > > > > > > > > >> >>>>>>>>>>>>> be.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think
> > > about
> > > >> > this
> > > >> > > > > trade
> > > >> > > > > > > > off.
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun
> Karimov
> > > >> wrote:<
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 to KIP.


<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>                              final Aggregator<? super K, ? super V, VR>
> adder,
> The adder doesn't need to be RichAggregator ?



- Exactly. However, there are 2 Aggregator-type arguments in the related
method. So, I had to overload all possible their Rich counterparts:

// adder with non-rich, subtrctor is rich
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final Aggregator<? super K, ? super V, VR>
adder,
                             final RichAggregator<? super K, ? super V, VR>
subtractor,
                             final Materialized<K, VR, KeyValueStore<Bytes,
byte[]>> materialized);

// adder withrich, subtrctor is non-rich
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final RichAggregator<? super K, ? super V, VR>
adder,
                             final Aggregator<? super K, ? super V, VR>
subtractor,
                             final Materialized<K, VR, KeyValueStore<Bytes,
byte[]>> materialized);

// both adder and subtractor are rich
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final RichAggregator<? super K, ? super V, VR>
adder,
                             final RichAggregator<? super K, ? super V, VR>
subtractor,
                             final Materialized<K, VR, KeyValueStore<Bytes,
byte[]>> materialized);


Can you explain a bit about the above implementation ?
>    void commit () {
>      throw new UnsupportedOperationException("commit() is not supported in
> this context");
> Is the exception going to be replaced with real code in the PR ?



- I added some comments both inside and outside the code snippets in KIP.
Specifically, for the code snippet above, we add *commit()* method to
*RecordContext* interface.
However, we want  *commit()* method to be used only for *RecordContext*
instances (at least for now), so we add UnsupportedOperationException in
all classes/interfaces that extend/implement *RecordContext.*
In general, 1) we make RecordContext publicly available within
ProcessorContext,  2) initialize its instance within all required
Processors and 3) pass it as an argument to the related Rich interfaces
inside Processors.




Cheers,
Jeyhun

On Fri, Sep 22, 2017 at 6:44 PM Ted Yu <yu...@gmail.com> wrote:

> 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.
>
> <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
>                              final Aggregator<? super K, ? super V, VR>
> adder,
>
> The adder doesn't need to be RichAggregator ?
>
>   public RecordContext recordContext() {
>     return this.recordContext();
>
> Can you explain a bit about the above implementation ?
>
>    void commit () {
>      throw new UnsupportedOperationException("commit() is not supported in
> this context");
>
> Is the exception going to be replaced with real code in the PR ?
>
> Cheers
>
>
> On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <je...@gmail.com>
> wrote:
>
> > 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 <je...@gmail.com>
> > wrote:
> >
> > > 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 <da...@gmail.com>
> 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 Jeyhun Karimov <je...@gmail.com>
> > wrote:
> > >>
> > >> > 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 merge this KIP into the upcoming release, as KIP-159 is not voted
> > yet
> > >> > (because we want both KIP-149 and KIP-159 to be as an "atomic"
> merge).
> > >> So
> > >> > I decided to wait until KIP-182 gets stable (there are some minor
> > >> updates
> > >> > AFAIK) and update the KIP accordingly. Please correct me if I am
> wrong
> > >> or I
> > >> > misunderstood.
> > >> >
> > >> > Cheers,
> > >> > Jeyhun
> > >> >
> > >> >
> > >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com>
> > >> wrote:
> > >> >
> > >> > > +1
> > >> > >
> > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com>
> > >> 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 would still
> > >> need to
> > >> > > > update your proposal for people to comment and vote on.
> > >> > > >
> > >> > > >
> > >> > > > Guozhang
> > >> > > >
> > >> > > >
> > >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com>
> > >> wrote:
> > >> > > >
> > >> > > > > +1
> > >> > > > >
> > >> > > > > One interface is cleaner.
> > >> > > > >
> > >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <
> bbejeck@gmail.com
> > >
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
> > >> > interfaces
> > >> > > > > into 1
> > >> > > > > > interface.
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Bill
> > >> > > > > >
> > >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > >> > > je.karimov@gmail.com
> > >> > > > >
> > >> > > > > > 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 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
> > >> > > > > > this
> > >> > > > > > > would not be a problem.
> > >> > > > > > >
> > >> > > > > > > Our initial idea was similar to :
> > >> > > > > > >
> > >> > > > > > > public abstract class RichValueMapper<K, V, VR>
> implements
> > >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > >> > > > > > > ......
> > >> > > > > > > }
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > So, we check the type of object, whether it is RichXXX or
> > >> > > XXXWithKey
> > >> > > > > > inside
> > >> > > > > > > the called method and continue accordingly.
> > >> > > > > > >
> > >> > > > > > > If this is ok with the community, I would like to revert
> the
> > >> > > current
> > >> > > > > > design
> > >> > > > > > > to this again.
> > >> > > > > > >
> > >> > > > > > > Cheers,
> > >> > > > > > > Jeyhun
> > >> > > > > > >
> > >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> > >> damian.guy@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hi Jeyhun,
> > >> > > > > > > >
> > >> > > > > > > > Thanks for sending out the update. I guess i was
> thinking
> > >> more
> > >> > > > along
> > >> > > > > > the
> > >> > > > > > > > lines of option 2 where we collapse the RichXXXX and
> > >> > > > ValueXXXXWithKey
> > >> > > > > > 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?
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Damian
> > >> > > > > > > >
> > >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > >> > > je.karimov@gmail.com>
> > >> > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > 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 free to comment on this.
> > >> > > > > > > > >
> > >> > > > > > > > > [1]
> > >> > > > > > > > >
> > >> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > >> > > > > > > action?pageId=73637757
> > >> > > > > > > > >
> > >> > > > > > > > > Cheers,
> > >> > > > > > > > > Jeyhun
> > >> > > > > > > > >
> > >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > >> > > > > je.karimov@gmail.com
> > >> > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > 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 <
> > >> > > > > > > matthias@confluent.io>
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > >> 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
> > >> > > > > > > > > >> > for one-shot solution for overloads issue.
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > Cheers,
> > >> > > > > > > > > >> > Jeyhun
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > >> > > > > > damian.guy@gmail.com>
> > >> > > > > > > > > >> wrote:
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> >> 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
> > >> > > > > > > > > overrides.
> > >> > > > > > > > > >> >>
> > >> > > > > > > > > >> >> Thanks,
> > >> > > > > > > > > >> >> Damian
> > >> > > > > > > > > >> >>
> > >> > > > > > > > > >> >>
> > >> > > > > > > > > >> >>> About ProcessorContext vs RecordContext, you
> are
> > >> > right.
> > >> > > I
> > >> > > > > > think
> > >> > > > > > > I
> > >> > > > > > > > > >> need to
> > >> > > > > > > > > >> >>> implement a prototype to understand the full
> > >> picture
> > >> > as
> > >> > > > some
> > >> > > > > > > parts
> > >> > > > > > > > > of
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> >>> KIP might not be as straightforward as I
> thought.
> > >> > > > > > > > > >> >>>
> > >> > > > > > > > > >> >>>
> > >> > > > > > > > > >> >>> Cheers,
> > >> > > > > > > > > >> >>> Jeyhun
> > >> > > > > > > > > >> >>>
> > >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > >> > > > > > > damian.guy@gmail.com>
> > >> > > > > > > > > >> wrote:
> > >> > > > > > > > > >> >>>
> > >> > > > > > > > > >> >>>> 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
> > >> > > > > lifetime
> > >> > > > > > of
> > >> > > > > > > > the
> > >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > >> > > > > > ProcessorContext
> > >> > > > > > > > to
> > >> > > > > > > > > a
> > >> > > > > > > > > >> >>>> RecordContext.
> > >> > > > > > > > > >> >>>> You mentioned above passing the
> > >> > > InternalProcessorContext
> > >> > > > to
> > >> > > > > > the
> > >> > > > > > > > > >> init()
> > >> > > > > > > > > >> >>>> calls. It is internal for a reason and i think
> > it
> > >> > > should
> > >> > > > > > remain
> > >> > > > > > > > > that
> > >> > > > > > > > > >> >> way.
> > >> > > > > > > > > >> >>>> It might be better to move the recordContext()
> > >> method
> > >> > > > from
> > >> > > > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > >> > > > > > > > > >> >>>>
> > >> > > > > > > > > >> >>>> In the KIP you have an example showing:
> > >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> > processorContext);
> > >> > > > > > > > > >> >>>> But the interface is:
> > >> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > >> > > > > > > > > >> >>>>     VR apply(final V value, final
> RecordContext
> > >> > > > > > recordContext);
> > >> > > > > > > > > >> >>>> }
> > >> > > > > > > > > >> >>>> i.e., there is no init(...), besides as above
> > this
> > >> > > > wouldn't
> > >> > > > > > > make
> > >> > > > > > > > > >> sense.
> > >> > > > > > > > > >> >>>>
> > >> > > > > > > > > >> >>>> Thanks,
> > >> > > > > > > > > >> >>>> Damian
> > >> > > > > > > > > >> >>>>
> > >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > >> > > > > > > je.karimov@gmail.com
> > >> > > > > > > > >
> > >> > > > > > > > > >> >> wrote:
> > >> > > > > > > > > >> >>>>
> > >> > > > > > > > > >> >>>>> 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.
> > >> Can't
> > >> > we
> > >> > > > pass
> > >> > > > > > > this
> > >> > > > > > > > > as a
> > >> > > > > > > > > >> >>>>> parameter to init() method of processors?
> Then
> > we
> > >> > > would
> > >> > > > be
> > >> > > > > > > able
> > >> > > > > > > > to
> > >> > > > > > > > > >> >> get
> > >> > > > > > > > > >> >>>>> RecordContext easily with just a method call.
> > >> > > > > > > > > >> >>>>>
> > >> > > > > > > > > >> >>>>>
> > >> > > > > > > > > >> >>>>> Cheers,
> > >> > > > > > > > > >> >>>>> Jeyhun
> > >> > > > > > > > > >> >>>>>
> > >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J.
> > Sax
> > >> <
> > >> > > > > > > > > >> >>> matthias@confluent.io>
> > >> > > > > > > > > >> >>>>> wrote:
> > >> > > > > > > > > >> >>>>>
> > >> > > > > > > > > >> >>>>>> 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
> > >> > > > > > > > > >> >>> will
> > >> > > > > > > > > >> >>>>>> be provided by the consecutive call to
> > aggregate
> > >> > > > anyways.
> > >> > > > > > > > > >> >>>>>>
> > >> > > > > > > > > >> >>>>>>
> > >> > > > > > > > > >> >>>>>> -Matthias
> > >> > > > > > > > > >> >>>>>>
> > >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > >> > > > > > > > > >> >>>>>>> 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 implement
> > >> both.
> > >> > > This
> > >> > > > > > allows
> > >> > > > > > > > us
> > >> > > > > > > > > >> >> to
> > >> > > > > > > > > >> >>>> cast
> > >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the
> > >> visible
> > >> > > > scope.
> > >> > > > > > > > > >> >>>>>>>
> > >> > > > > > > > > >> >>>>>>>
> > >> > > > > > > > > >> >>>>>>> -Matthias
> > >> > > > > > > > > >> >>>>>>>
> > >> > > > > > > > > >> >>>>>>>
> > >> > > > > > > > > >> >>>>>>>
> > >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > >> > > > > > > > > >> >>>>>>>> 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 address this issue as well
> or
> > >> > > > > > > > > >> >>>>>>>> continue as it is?
> > >> > > > > > > > > >> >>>>>>>>
> > >> > > > > > > > > >> >>>>>>>> Cheers,
> > >> > > > > > > > > >> >>>>>>>> Jeyhun
> > >> > > > > > > > > >> >>>>>>>>
> > >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang
> > >> Wang <
> > >> > > > > > > > > >> >>> wangguoz@gmail.com>
> > >> > > > > > > > > >> >>>>>> wrote:
> > >> > > > > > > > > >> >>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > >> > > > > > > > > >> >>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>> Guozhang
> > >> > > > > > > > > >> >>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun
> > >> Karimov
> > >> > <
> > >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > >> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1]
> > can
> > >> > also
> > >> > > > be
> > >> > > > > > > > > >> >> integrated
> > >> > > > > > > > > >> >>>> into
> > >> > > > > > > > > >> >>>>>> the
> > >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> > >> > RecordContext
> > >> > > > > > > > interface.
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>> [1]
> > >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>> Cheers,
> > >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias
> > J.
> > >> > Sax <
> > >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > >> > > > > > > > > >> >>>>>>>>>> wrote:
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> 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
> > information
> > >> > only.
> > >> > > > If
> > >> > > > > > you
> > >> > > > > > > > want
> > >> > > > > > > > > >> >> to
> > >> > > > > > > > > >> >>>> do
> > >> > > > > > > > > >> >>>>> a
> > >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should rather
> > use
> > >> > > > > > #transform().
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would
> > need
> > >> to
> > >> > > > switch
> > >> > > > > > to
> > >> > > > > > > a
> > >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
> > >> > > overloads.
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a
> simple
> > >> > > > interface
> > >> > > > > > > with a
> > >> > > > > > > > > >> >>> single
> > >> > > > > > > > > >> >>>>>>>>> method
> > >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > >> > > > > > > > > >> >>>>>>>>>>>> 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
> > >> > > <
> > >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > >> > > > > > > > > >> >>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>> wrote:
> > >> > > > > > > > > >> >>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will not
> > >> mention
> > >> > > > them.
> > >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in
> > >> > KIP-149
> > >> > > > and
> > >> > > > > in
> > >> > > > > > > > this
> > >> > > > > > > > > >> >> KIP
> > >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came
> into
> > >> the
> > >> > > > stage
> > >> > > > > > with
> > >> > > > > > > > > >> >>> KIP-149,
> > >> > > > > > > > > >> >>>>> in
> > >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we
> > >> extended
> > >> > > > KIP-149
> > >> > > > > > to
> > >> > > > > > > > > >> >> support
> > >> > > > > > > > > >> >>>>> Rich
> > >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions,
> > we
> > >> > > > provided
> > >> > > > > > init
> > >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested
> > >> that
> > >> > we
> > >> > > > > should
> > >> > > > > > > not
> > >> > > > > > > > > >> >>>> provide
> > >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a
> result,
> > >> we
> > >> > > > > separated
> > >> > > > > > > the
> > >> > > > > > > > > >> >> two
> > >> > > > > > > > > >> >>>>>>>>> problems
> > >> > > > > > > > > >> >>>>>>>>>>> into
> > >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they
> can
> > >> be
> > >> > > > solved
> > >> > > > > in
> > >> > > > > > > > > >> >>> parallel.
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> public interface
> ValueMapperWithKey<K,
> > V,
> > >> > VR>
> > >> > > {
> > >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V
> > value);
> > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K,
> V,
> > >> VR>
> > >> > > > extends
> > >> > > > > > > > > >> >>>> RichFunction{
> > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> > >> recordContext);
> > >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > >> > > > > appConfigsWithPrefix(String
> > >> > > > > > > > > >> >> prefix);
> > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext
> > extends
> > >> > > > > > > RecordContext
> > >> > > > > > > > {
> > >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> > >> > > RecordContext
> > >> > > > > > > > > >> >>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
> > >> > interfaces
> > >> > > > can
> > >> > > > > be
> > >> > > > > > > > > >> >>> converted
> > >> > > > > > > > > >> >>>> to
> > >> > > > > > > > > >> >>>>>>>>>> their
> > >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init()
> > and
> > >> > > close()
> > >> > > > > > > > methods)
> > >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept
> > >> Rich
> > >> > > > > > interfaces
> > >> > > > > > > in
> > >> > > > > > > > > >> >>> their
> > >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related
> "withKey"
> > >> or
> > >> > > > > > > "withoutKey"
> > >> > > > > > > > > >> >>>>> interfaces
> > >> > > > > > > > > >> >>>>>>>>> to
> > >> > > > > > > > > >> >>>>>>>>>>> Rich
> > >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the topology
> > and
> > >> > > > > initialize
> > >> > > > > > > the
> > >> > > > > > > > > >> >>>> related
> > >> > > > > > > > > >> >>>>>>>>>>> processors
> > >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded
> > methods
> > >> > for
> > >> > > > rich
> > >> > > > > > > > > >> >> functions
> > >> > > > > > > > > >> >>> as
> > >> > > > > > > > > >> >>>>>> Rich
> > >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces.
> > We
> > >> > will
> > >> > > > just
> > >> > > > > > > check
> > >> > > > > > > > > >> >> the
> > >> > > > > > > > > >> >>>>> object
> > >> > > > > > > > > >> >>>>>>>>>> type
> > >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the
> > above
> > >> > > > approach
> > >> > > > > > > does
> > >> > > > > > > > > not
> > >> > > > > > > > > >> >>>>> support
> > >> > > > > > > > > >> >>>>>>>>>>> lambdas
> > >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one method,
> > >> only
> > >> > > > > > > > > >> >>> init(RecordContext),
> > >> > > > > > > > > >> >>>>> as
> > >> > > > > > > > > >> >>>>>>>>>> part
> > >> > > > > > > > > >> >>>>>>>>>>> of
> > >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> > Personally I
> > >> > > think
> > >> > > > > Rich
> > >> > > > > > > > > >> >>> interfaces
> > >> > > > > > > > > >> >>>>> are
> > >> > > > > > > > > >> >>>>>>>>> by
> > >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should
> > not
> > >> > care
> > >> > > > much
> > >> > > > > > > about
> > >> > > > > > > > > >> >> it.
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion,
> an
> > >> > > > alternative
> > >> > > > > > we
> > >> > > > > > > > > >> >>>> considered
> > >> > > > > > > > > >> >>>>>> was
> > >> > > > > > > > > >> >>>>>>>>>> to
> > >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
> > >> > parameter.
> > >> > > > > This
> > >> > > > > > > > might
> > >> > > > > > > > > >> >>> even
> > >> > > > > > > > > >> >>>>>>>>> allow
> > >> > > > > > > > > >> >>>>>>>>>> to
> > >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the
> name
> > >> > > > > RichFunction
> > >> > > > > > as
> > >> > > > > > > > we
> > >> > > > > > > > > >> >>>>> preserve
> > >> > > > > > > > > >> >>>>>>>>> the
> > >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and
> `close()`
> > we
> > >> > > > > basically
> > >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory
> state
> > >> for a
> > >> > > > > > function.
> > >> > > > > > > > > >> >> Thus,
> > >> > > > > > > > > >> >>> we
> > >> > > > > > > > > >> >>>>>>>>> cannot
> > >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> > >> RichValueMapper
> > >> > > (etc)
> > >> > > > > > over
> > >> > > > > > > > > >> >>> multiple
> > >> > > > > > > > > >> >>>>>> tasks
> > >> > > > > > > > > >> >>>>>>>>>> and
> > >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern
> > similar
> > >> to
> > >> > > > > > > > #transform().
> > >> > > > > > > > > >> >> And
> > >> > > > > > > > > >> >>>>> this
> > >> > > > > > > > > >> >>>>>>>>>> would
> > >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > >> > > > > > > > (Rich)ValueMapperSupplier
> > >> > > > > > > > > >> >>> would
> > >> > > > > > > > > >> >>>>> not
> > >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we
> > >> would
> > >> > > need
> > >> > > > > many
> > >> > > > > > > new
> > >> > > > > > > > > >> >>>> overload
> > >> > > > > > > > > >> >>>>>>>>> for
> > >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste
> > from
> > >> > > > > Matthias's
> > >> > > > > > > > > email)
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM
> Matthias
> > >> J.
> > >> > > Sax <
> > >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > >> > > > > > > > > >> >>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > >> > > > > > > > > >> >>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > >> > > > 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:
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> > >> > RecordContext
> > >> > > > > > > context);
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > >> > mapValues(ValueMapper<?
> > >> > > > > super
> > >> > > > > > > V, ?
> > >> > > > > > > > > >> >>>> extends
> > >> > > > > > > > > >> >>>>>> VR>
> > >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > >> > > > > > > > mapValueswithContext(RichValueMapper
> > >> > > > > > > > > >> >> <?
> > >> > > > > > > > > >> >>>>> super
> > >> > > > > > > > > >> >>>>>>>>>> V, ?
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will
> introduces
> > >> more
> > >> > > > > > > overloads;
> > >> > > > > > > > > >> >> but I
> > >> > > > > > > > > >> >>>>> think
> > >> > > > > > > > > >> >>>>>>>>>> the
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced
> by
> > 1)
> > >> > > serde
> > >> > > > > > > > overrides
> > >> > > > > > > > > >> >> and
> > >> > > > > > > > > >> >>>> 2)
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both
> > of
> > >> > which
> > >> > > > can
> > >> > > > > > be
> > >> > > > > > > > > >> >> reduced
> > >> > > > > > > > > >> >>>> in
> > >> > > > > > > > > >> >>>>>> the
> > >> > > > > > > > > >> >>>>>>>>>>> near
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading
> > is
> > >> > still
> > >> > > > > > > > worthwhile,
> > >> > > > > > > > > >> >> as
> > >> > > > > > > > > >> >>>> it
> > >> > > > > > > > > >> >>>>>> has
> > >> > > > > > > > > >> >>>>>>>>>> the
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need
> to
> > >> > > "convert"
> > >> > > > > > from
> > >> > > > > > > > > >> >>> non-rich
> > >> > > > > > > > > >> >>>>>>>>>> functions
> > >> > > > > > > > > >> >>>>>>>>>>>>> to
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already
> been
> > >> > > discussed
> > >> > > > > > and I
> > >> > > > > > > > may
> > >> > > > > > > > > >> >>> have
> > >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > >> > > > > > > > > >> >>>>>>>>>>>>>> in
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM,
> > >> Matthias
> > >> > J.
> > >> > > > > Sax <
> > >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to
> > converge
> > >> to
> > >> > an
> > >> > > > > > overall
> > >> > > > > > > > > >> >>>> consisted
> > >> > > > > > > > > >> >>>>>>>>> API.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes,
> > we
> > >> > might
> > >> > > > get
> > >> > > > > > > more
> > >> > > > > > > > > >> >>>> overload.
> > >> > > > > > > > > >> >>>>>> It
> > >> > > > > > > > > >> >>>>>>>>>>>>> might
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a
> single
> > >> > > > > > > xxxWithContext()
> > >> > > > > > > > > >> >>>> overload
> > >> > > > > > > > > >> >>>>>>>>> that
> > >> > > > > > > > > >> >>>>>>>>>>>>> will
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> > Otherwise,
> > >> if
> > >> > > > might
> > >> > > > > > get
> > >> > > > > > > > too
> > >> > > > > > > > > >> >>> messy
> > >> > > > > > > > > >> >>>>>>>>> having
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > >> > > > > > > > > ValueMapperWithContext,
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have
> the
> > >> > > "builder
> > >> > > > > > > pattern"
> > >> > > > > > > > > >> >> idea
> > >> > > > > > > > > >> >>>> as
> > >> > > > > > > > > >> >>>>> an
> > >> > > > > > > > > >> >>>>>>>>>> API
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the
> > >> > overload
> > >> > > > > > > problem.
> > >> > > > > > > > > Not
> > >> > > > > > > > > >> >>> for
> > >> > > > > > > > > >> >>>>>>>>> simple
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but
> > for
> > >> > joins
> > >> > > > and
> > >> > > > > > > > > >> >>>> aggregations.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned
> in
> > >> an
> > >> > > older
> > >> > > > > > > email,
> > >> > > > > > > > I
> > >> > > > > > > > > >> >> am
> > >> > > > > > > > > >> >>>>>>>>>> personally
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
> > >> > > interface,
> > >> > > > > and
> > >> > > > > > > add
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext
> with
> > >> > method
> > >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > >> > > > > > > > > >> >>>>>>>>>> (or
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name)
> --
> > >> but
> > >> > not
> > >> > > > > > > > `close()`)
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > >> > ValueMapperWithRecordContext
> > >> > > > > > extends
> > >> > > > > > > > > >> >>>>> ValueMapper,
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
> > >> > overload.
> > >> > > Of
> > >> > > > > > > course,
> > >> > > > > > > > > we
> > >> > > > > > > > > >> >>>> don't
> > >> > > > > > > > > >> >>>>>>>>> get
> > >> > > > > > > > > >> >>>>>>>>>> a
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
> > >> > > sacrifices
> > >> > > > > > > > Lambdas.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit
> > undecided
> > >> > what
> > >> > > > the
> > >> > > > > > > > better
> > >> > > > > > > > > >> >>>> option
> > >> > > > > > > > > >> >>>>>>>>> might
> > >> > > > > > > > > >> >>>>>>>>>>>>> be.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think
> > about
> > >> > this
> > >> > > > > trade
> > >> > > > > > > > off.
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov
> > >> wrote:<

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

Posted by Ted Yu <yu...@gmail.com>.
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.

<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final Aggregator<? super K, ? super V, VR>
adder,

The adder doesn't need to be RichAggregator ?

  public RecordContext recordContext() {
    return this.recordContext();

Can you explain a bit about the above implementation ?

   void commit () {
     throw new UnsupportedOperationException("commit() is not supported in
this context");

Is the exception going to be replaced with real code in the PR ?

Cheers


On Fri, Sep 22, 2017 at 9:28 AM, Jeyhun Karimov <je...@gmail.com>
wrote:

> 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 <je...@gmail.com>
> wrote:
>
> > 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 <da...@gmail.com> 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 Jeyhun Karimov <je...@gmail.com>
> wrote:
> >>
> >> > 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 merge this KIP into the upcoming release, as KIP-159 is not voted
> yet
> >> > (because we want both KIP-149 and KIP-159 to be as an "atomic" merge).
> >> So
> >> > I decided to wait until KIP-182 gets stable (there are some minor
> >> updates
> >> > AFAIK) and update the KIP accordingly. Please correct me if I am wrong
> >> or I
> >> > misunderstood.
> >> >
> >> > Cheers,
> >> > Jeyhun
> >> >
> >> >
> >> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com>
> >> wrote:
> >> >
> >> > > +1
> >> > >
> >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com>
> >> 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 would still
> >> need to
> >> > > > update your proposal for people to comment and vote on.
> >> > > >
> >> > > >
> >> > > > Guozhang
> >> > > >
> >> > > >
> >> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > +1
> >> > > > >
> >> > > > > One interface is cleaner.
> >> > > > >
> >> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bbejeck@gmail.com
> >
> >> > > wrote:
> >> > > > >
> >> > > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
> >> > interfaces
> >> > > > > into 1
> >> > > > > > interface.
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Bill
> >> > > > > >
> >> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> >> > > je.karimov@gmail.com
> >> > > > >
> >> > > > > > 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 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
> >> > > > > > this
> >> > > > > > > would not be a problem.
> >> > > > > > >
> >> > > > > > > Our initial idea was similar to :
> >> > > > > > >
> >> > > > > > > public abstract class RichValueMapper<K, V, VR>  implements
> >> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> >> > > > > > > ......
> >> > > > > > > }
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > So, we check the type of object, whether it is RichXXX or
> >> > > XXXWithKey
> >> > > > > > inside
> >> > > > > > > the called method and continue accordingly.
> >> > > > > > >
> >> > > > > > > If this is ok with the community, I would like to revert the
> >> > > current
> >> > > > > > design
> >> > > > > > > to this again.
> >> > > > > > >
> >> > > > > > > Cheers,
> >> > > > > > > Jeyhun
> >> > > > > > >
> >> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> >> damian.guy@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hi Jeyhun,
> >> > > > > > > >
> >> > > > > > > > Thanks for sending out the update. I guess i was thinking
> >> more
> >> > > > along
> >> > > > > > the
> >> > > > > > > > lines of option 2 where we collapse the RichXXXX and
> >> > > > ValueXXXXWithKey
> >> > > > > > 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?
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Damian
> >> > > > > > > >
> >> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> >> > > je.karimov@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > 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 free to comment on this.
> >> > > > > > > > >
> >> > > > > > > > > [1]
> >> > > > > > > > >
> >> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> >> > > > > > > action?pageId=73637757
> >> > > > > > > > >
> >> > > > > > > > > Cheers,
> >> > > > > > > > > Jeyhun
> >> > > > > > > > >
> >> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> >> > > > > je.karimov@gmail.com
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > 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 <
> >> > > > > > > matthias@confluent.io>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > >> 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
> >> > > > > > > > > >> > for one-shot solution for overloads issue.
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > Cheers,
> >> > > > > > > > > >> > Jeyhun
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> >> > > > > > damian.guy@gmail.com>
> >> > > > > > > > > >> wrote:
> >> > > > > > > > > >> >
> >> > > > > > > > > >> >> 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
> >> > > > > > > > > overrides.
> >> > > > > > > > > >> >>
> >> > > > > > > > > >> >> Thanks,
> >> > > > > > > > > >> >> Damian
> >> > > > > > > > > >> >>
> >> > > > > > > > > >> >>
> >> > > > > > > > > >> >>> About ProcessorContext vs RecordContext, you are
> >> > right.
> >> > > I
> >> > > > > > think
> >> > > > > > > I
> >> > > > > > > > > >> need to
> >> > > > > > > > > >> >>> implement a prototype to understand the full
> >> picture
> >> > as
> >> > > > some
> >> > > > > > > parts
> >> > > > > > > > > of
> >> > > > > > > > > >> the
> >> > > > > > > > > >> >>> KIP might not be as straightforward as I thought.
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>> Cheers,
> >> > > > > > > > > >> >>> Jeyhun
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> >> > > > > > > damian.guy@gmail.com>
> >> > > > > > > > > >> wrote:
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>>> 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
> >> > > > > lifetime
> >> > > > > > of
> >> > > > > > > > the
> >> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> >> > > > > > ProcessorContext
> >> > > > > > > > to
> >> > > > > > > > > a
> >> > > > > > > > > >> >>>> RecordContext.
> >> > > > > > > > > >> >>>> You mentioned above passing the
> >> > > InternalProcessorContext
> >> > > > to
> >> > > > > > the
> >> > > > > > > > > >> init()
> >> > > > > > > > > >> >>>> calls. It is internal for a reason and i think
> it
> >> > > should
> >> > > > > > remain
> >> > > > > > > > > that
> >> > > > > > > > > >> >> way.
> >> > > > > > > > > >> >>>> It might be better to move the recordContext()
> >> method
> >> > > > from
> >> > > > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> >> > > > > > > > > >> >>>>
> >> > > > > > > > > >> >>>> In the KIP you have an example showing:
> >> > > > > > > > > >> >>>> richMapper.init((RecordContext)
> processorContext);
> >> > > > > > > > > >> >>>> But the interface is:
> >> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> >> > > > > > > > > >> >>>>     VR apply(final V value, final RecordContext
> >> > > > > > recordContext);
> >> > > > > > > > > >> >>>> }
> >> > > > > > > > > >> >>>> i.e., there is no init(...), besides as above
> this
> >> > > > wouldn't
> >> > > > > > > make
> >> > > > > > > > > >> sense.
> >> > > > > > > > > >> >>>>
> >> > > > > > > > > >> >>>> Thanks,
> >> > > > > > > > > >> >>>> Damian
> >> > > > > > > > > >> >>>>
> >> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> >> > > > > > > je.karimov@gmail.com
> >> > > > > > > > >
> >> > > > > > > > > >> >> wrote:
> >> > > > > > > > > >> >>>>
> >> > > > > > > > > >> >>>>> 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.
> >> Can't
> >> > we
> >> > > > pass
> >> > > > > > > this
> >> > > > > > > > > as a
> >> > > > > > > > > >> >>>>> parameter to init() method of processors? Then
> we
> >> > > would
> >> > > > be
> >> > > > > > > able
> >> > > > > > > > to
> >> > > > > > > > > >> >> get
> >> > > > > > > > > >> >>>>> RecordContext easily with just a method call.
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>> Cheers,
> >> > > > > > > > > >> >>>>> Jeyhun
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J.
> Sax
> >> <
> >> > > > > > > > > >> >>> matthias@confluent.io>
> >> > > > > > > > > >> >>>>> wrote:
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>>> 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
> >> > > > > > > > > >> >>> will
> >> > > > > > > > > >> >>>>>> be provided by the consecutive call to
> aggregate
> >> > > > anyways.
> >> > > > > > > > > >> >>>>>>
> >> > > > > > > > > >> >>>>>>
> >> > > > > > > > > >> >>>>>> -Matthias
> >> > > > > > > > > >> >>>>>>
> >> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> >> > > > > > > > > >> >>>>>>> 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 implement
> >> both.
> >> > > This
> >> > > > > > allows
> >> > > > > > > > us
> >> > > > > > > > > >> >> to
> >> > > > > > > > > >> >>>> cast
> >> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the
> >> visible
> >> > > > scope.
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>> -Matthias
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> >> > > > > > > > > >> >>>>>>>> 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 address this issue as well or
> >> > > > > > > > > >> >>>>>>>> continue as it is?
> >> > > > > > > > > >> >>>>>>>>
> >> > > > > > > > > >> >>>>>>>> Cheers,
> >> > > > > > > > > >> >>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>
> >> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang
> >> Wang <
> >> > > > > > > > > >> >>> wangguoz@gmail.com>
> >> > > > > > > > > >> >>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>
> >> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>> Guozhang
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun
> >> Karimov
> >> > <
> >> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> >> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1]
> can
> >> > also
> >> > > > be
> >> > > > > > > > > >> >> integrated
> >> > > > > > > > > >> >>>> into
> >> > > > > > > > > >> >>>>>> the
> >> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> >> > RecordContext
> >> > > > > > > > interface.
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>> [1]
> >> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>> Cheers,
> >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias
> J.
> >> > Sax <
> >> > > > > > > > > >> >>>>>> matthias@confluent.io>
> >> > > > > > > > > >> >>>>>>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> 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
> information
> >> > only.
> >> > > > If
> >> > > > > > you
> >> > > > > > > > want
> >> > > > > > > > > >> >> to
> >> > > > > > > > > >> >>>> do
> >> > > > > > > > > >> >>>>> a
> >> > > > > > > > > >> >>>>>>>>>>> stateful computation you should rather
> use
> >> > > > > > #transform().
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would
> need
> >> to
> >> > > > switch
> >> > > > > > to
> >> > > > > > > a
> >> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
> >> > > overloads.
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
> >> > > > interface
> >> > > > > > > with a
> >> > > > > > > > > >> >>> single
> >> > > > > > > > > >> >>>>>>>>> method
> >> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> -Matthias
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> >> > > > > > > > > >> >>>>>>>>>>>> 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
> >> > > <
> >> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will not
> >> mention
> >> > > > them.
> >> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in
> >> > KIP-149
> >> > > > and
> >> > > > > in
> >> > > > > > > > this
> >> > > > > > > > > >> >> KIP
> >> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> >> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into
> >> the
> >> > > > stage
> >> > > > > > with
> >> > > > > > > > > >> >>> KIP-149,
> >> > > > > > > > > >> >>>>> in
> >> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we
> >> extended
> >> > > > KIP-149
> >> > > > > > to
> >> > > > > > > > > >> >> support
> >> > > > > > > > > >> >>>>> Rich
> >> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions,
> we
> >> > > > provided
> >> > > > > > init
> >> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> >> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested
> >> that
> >> > we
> >> > > > > should
> >> > > > > > > not
> >> > > > > > > > > >> >>>> provide
> >> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result,
> >> we
> >> > > > > separated
> >> > > > > > > the
> >> > > > > > > > > >> >> two
> >> > > > > > > > > >> >>>>>>>>> problems
> >> > > > > > > > > >> >>>>>>>>>>> into
> >> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can
> >> be
> >> > > > solved
> >> > > > > in
> >> > > > > > > > > >> >>> parallel.
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K,
> V,
> >> > VR>
> >> > > {
> >> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V
> value);
> >> > > > > > > > > >> >>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V,
> >> VR>
> >> > > > extends
> >> > > > > > > > > >> >>>> RichFunction{
> >> > > > > > > > > >> >>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> >> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> >> recordContext);
> >> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> >> > > > > > > > > >> >>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> >> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> >> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> >> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> >> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> >> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> >> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> >> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> >> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> >> > > > > appConfigsWithPrefix(String
> >> > > > > > > > > >> >> prefix);
> >> > > > > > > > > >> >>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext
> extends
> >> > > > > > > RecordContext
> >> > > > > > > > {
> >> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> >> > > RecordContext
> >> > > > > > > > > >> >>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> >> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
> >> > interfaces
> >> > > > can
> >> > > > > be
> >> > > > > > > > > >> >>> converted
> >> > > > > > > > > >> >>>> to
> >> > > > > > > > > >> >>>>>>>>>> their
> >> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init()
> and
> >> > > close()
> >> > > > > > > > methods)
> >> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept
> >> Rich
> >> > > > > > interfaces
> >> > > > > > > in
> >> > > > > > > > > >> >>> their
> >> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> >> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey"
> >> or
> >> > > > > > > "withoutKey"
> >> > > > > > > > > >> >>>>> interfaces
> >> > > > > > > > > >> >>>>>>>>> to
> >> > > > > > > > > >> >>>>>>>>>>> Rich
> >> > > > > > > > > >> >>>>>>>>>>>>> interface while building the topology
> and
> >> > > > > initialize
> >> > > > > > > the
> >> > > > > > > > > >> >>>> related
> >> > > > > > > > > >> >>>>>>>>>>> processors
> >> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> >> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded
> methods
> >> > for
> >> > > > rich
> >> > > > > > > > > >> >> functions
> >> > > > > > > > > >> >>> as
> >> > > > > > > > > >> >>>>>> Rich
> >> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces.
> We
> >> > will
> >> > > > just
> >> > > > > > > check
> >> > > > > > > > > >> >> the
> >> > > > > > > > > >> >>>>> object
> >> > > > > > > > > >> >>>>>>>>>> type
> >> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the
> above
> >> > > > approach
> >> > > > > > > does
> >> > > > > > > > > not
> >> > > > > > > > > >> >>>>> support
> >> > > > > > > > > >> >>>>>>>>>>> lambdas
> >> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one method,
> >> only
> >> > > > > > > > > >> >>> init(RecordContext),
> >> > > > > > > > > >> >>>>> as
> >> > > > > > > > > >> >>>>>>>>>> part
> >> > > > > > > > > >> >>>>>>>>>>> of
> >> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> >> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion.
> Personally I
> >> > > think
> >> > > > > Rich
> >> > > > > > > > > >> >>> interfaces
> >> > > > > > > > > >> >>>>> are
> >> > > > > > > > > >> >>>>>>>>> by
> >> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should
> not
> >> > care
> >> > > > much
> >> > > > > > > about
> >> > > > > > > > > >> >> it.
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
> >> > > > alternative
> >> > > > > > we
> >> > > > > > > > > >> >>>> considered
> >> > > > > > > > > >> >>>>>> was
> >> > > > > > > > > >> >>>>>>>>>> to
> >> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
> >> > parameter.
> >> > > > > This
> >> > > > > > > > might
> >> > > > > > > > > >> >>> even
> >> > > > > > > > > >> >>>>>>>>> allow
> >> > > > > > > > > >> >>>>>>>>>> to
> >> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> >> > > > > RichFunction
> >> > > > > > as
> >> > > > > > > > we
> >> > > > > > > > > >> >>>>> preserve
> >> > > > > > > > > >> >>>>>>>>> the
> >> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> >> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()`
> we
> >> > > > > basically
> >> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state
> >> for a
> >> > > > > > function.
> >> > > > > > > > > >> >> Thus,
> >> > > > > > > > > >> >>> we
> >> > > > > > > > > >> >>>>>>>>> cannot
> >> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
> >> RichValueMapper
> >> > > (etc)
> >> > > > > > over
> >> > > > > > > > > >> >>> multiple
> >> > > > > > > > > >> >>>>>> tasks
> >> > > > > > > > > >> >>>>>>>>>> and
> >> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern
> similar
> >> to
> >> > > > > > > > #transform().
> >> > > > > > > > > >> >> And
> >> > > > > > > > > >> >>>>> this
> >> > > > > > > > > >> >>>>>>>>>> would
> >> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> >> > > > > > > > (Rich)ValueMapperSupplier
> >> > > > > > > > > >> >>> would
> >> > > > > > > > > >> >>>>> not
> >> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we
> >> would
> >> > > need
> >> > > > > many
> >> > > > > > > new
> >> > > > > > > > > >> >>>> overload
> >> > > > > > > > > >> >>>>>>>>> for
> >> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste
> from
> >> > > > > Matthias's
> >> > > > > > > > > email)
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias
> >> J.
> >> > > Sax <
> >> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>> 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
> >> > > > 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:
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> >> > RecordContext
> >> > > > > > > context);
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> }
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> >> > mapValues(ValueMapper<?
> >> > > > > super
> >> > > > > > > V, ?
> >> > > > > > > > > >> >>>> extends
> >> > > > > > > > > >> >>>>>> VR>
> >> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> >> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> >> > > > > > > > mapValueswithContext(RichValueMapper
> >> > > > > > > > > >> >> <?
> >> > > > > > > > > >> >>>>> super
> >> > > > > > > > > >> >>>>>>>>>> V, ?
> >> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces
> >> more
> >> > > > > > > overloads;
> >> > > > > > > > > >> >> but I
> >> > > > > > > > > >> >>>>> think
> >> > > > > > > > > >> >>>>>>>>>> the
> >> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by
> 1)
> >> > > serde
> >> > > > > > > > overrides
> >> > > > > > > > > >> >> and
> >> > > > > > > > > >> >>>> 2)
> >> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both
> of
> >> > which
> >> > > > can
> >> > > > > > be
> >> > > > > > > > > >> >> reduced
> >> > > > > > > > > >> >>>> in
> >> > > > > > > > > >> >>>>>> the
> >> > > > > > > > > >> >>>>>>>>>>> near
> >> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading
> is
> >> > still
> >> > > > > > > > worthwhile,
> >> > > > > > > > > >> >> as
> >> > > > > > > > > >> >>>> it
> >> > > > > > > > > >> >>>>>> has
> >> > > > > > > > > >> >>>>>>>>>> the
> >> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> >> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to
> >> > > "convert"
> >> > > > > > from
> >> > > > > > > > > >> >>> non-rich
> >> > > > > > > > > >> >>>>>>>>>> functions
> >> > > > > > > > > >> >>>>>>>>>>>>> to
> >> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been
> >> > > discussed
> >> > > > > > and I
> >> > > > > > > > may
> >> > > > > > > > > >> >>> have
> >> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> >> > > > > > > > > >> >>>>>>>>>>>>>> in
> >> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM,
> >> Matthias
> >> > J.
> >> > > > > Sax <
> >> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> >> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to
> converge
> >> to
> >> > an
> >> > > > > > overall
> >> > > > > > > > > >> >>>> consisted
> >> > > > > > > > > >> >>>>>>>>> API.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes,
> we
> >> > might
> >> > > > get
> >> > > > > > > more
> >> > > > > > > > > >> >>>> overload.
> >> > > > > > > > > >> >>>>>> It
> >> > > > > > > > > >> >>>>>>>>>>>>> might
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> >> > > > > > > xxxWithContext()
> >> > > > > > > > > >> >>>> overload
> >> > > > > > > > > >> >>>>>>>>> that
> >> > > > > > > > > >> >>>>>>>>>>>>> will
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context.
> Otherwise,
> >> if
> >> > > > might
> >> > > > > > get
> >> > > > > > > > too
> >> > > > > > > > > >> >>> messy
> >> > > > > > > > > >> >>>>>>>>> having
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> >> > > > > > > > > ValueMapperWithContext,
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the
> >> > > "builder
> >> > > > > > > pattern"
> >> > > > > > > > > >> >> idea
> >> > > > > > > > > >> >>>> as
> >> > > > > > > > > >> >>>>> an
> >> > > > > > > > > >> >>>>>>>>>> API
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the
> >> > overload
> >> > > > > > > problem.
> >> > > > > > > > > Not
> >> > > > > > > > > >> >>> for
> >> > > > > > > > > >> >>>>>>>>> simple
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but
> for
> >> > joins
> >> > > > and
> >> > > > > > > > > >> >>>> aggregations.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in
> >> an
> >> > > older
> >> > > > > > > email,
> >> > > > > > > > I
> >> > > > > > > > > >> >> am
> >> > > > > > > > > >> >>>>>>>>>> personally
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
> >> > > interface,
> >> > > > > and
> >> > > > > > > add
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with
> >> > method
> >> > > > > > > > > >> >>>>>> `open(RecordContext)`
> >> > > > > > > > > >> >>>>>>>>>> (or
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) --
> >> but
> >> > not
> >> > > > > > > > `close()`)
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> >> > ValueMapperWithRecordContext
> >> > > > > > extends
> >> > > > > > > > > >> >>>>> ValueMapper,
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
> >> > overload.
> >> > > Of
> >> > > > > > > course,
> >> > > > > > > > > we
> >> > > > > > > > > >> >>>> don't
> >> > > > > > > > > >> >>>>>>>>> get
> >> > > > > > > > > >> >>>>>>>>>> a
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
> >> > > sacrifices
> >> > > > > > > > Lambdas.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit
> undecided
> >> > what
> >> > > > the
> >> > > > > > > > better
> >> > > > > > > > > >> >>>> option
> >> > > > > > > > > >> >>>>>>>>> might
> >> > > > > > > > > >> >>>>>>>>>>>>> be.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think
> about
> >> > this
> >> > > > > trade
> >> > > > > > > > off.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov
> >> wrote:
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>> 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.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM
> >> Guozhang
> >> > > > Wang <
> >> > > > > > > > > >> >>>>>>>>> wangguoz@gmail.com>
> >> > > > > > > > > >> >>>>>>>>>>>>>> wrote:
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> >> > > > > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your
> comments.
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>
> >> > > > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> >> > > > > functions+to+Streams
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> --
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>> --
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>>> --
> >> > > > > > > > > >> >>>>>>>>>>>>> -Cheers
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>>> --
> >> > > > > > > > > >> >>>>>>>>>> -Cheers
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>> Jeyhun
> >> > > > > > > > > >> >>>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>>> --
> >> > > > > > > > > >> >>>>>>>>> -- Guozhang
> >> > > > > > > > > >> >>>>>>>>>
> >> > > > > > > > > >> >>>>>>>
> >> > > > > > > > > >> >>>>>>
> >> > > > > > > > > >> >>>>>> --
> >> > > > > > > > > >> >>>>> -Cheers
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>> Jeyhun
> >> > > > > > > > > >> >>>>>
> >> > > > > > > > > >> >>>>
> >> > > > > > > > > >> >>> --
> >> > > > > > > > > >> >>> -Cheers
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>> Jeyhun
> >> > > > > > > > > >> >>>
> >> > > > > > > > > >> >>
> >> > > > > > > > > >>
> >> > > > > > > > > >> --
> >> > > > > > > > > > -Cheers
> >> > > > > > > > > >
> >> > > > > > > > > > Jeyhun
> >> > > > > > > > > >
> >> > > > > > > > > --
> >> > > > > > > > > -Cheers
> >> > > > > > > > >
> >> > > > > > > > > Jeyhun
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > > --
> >> > > > > > > -Cheers
> >> > > > > > >
> >> > > > > > > Jeyhun
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > --
> >> > > > -- Guozhang
> >> > > >
> >> > >
> >> >
> >>
> >
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 <je...@gmail.com>
wrote:

> 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 <da...@gmail.com> 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 Jeyhun Karimov <je...@gmail.com> wrote:
>>
>> > 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 merge this KIP into the upcoming release, as KIP-159 is not voted yet
>> > (because we want both KIP-149 and KIP-159 to be as an "atomic" merge).
>> So
>> > I decided to wait until KIP-182 gets stable (there are some minor
>> updates
>> > AFAIK) and update the KIP accordingly. Please correct me if I am wrong
>> or I
>> > misunderstood.
>> >
>> > Cheers,
>> > Jeyhun
>> >
>> >
>> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com>
>> wrote:
>> >
>> > > +1
>> > >
>> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com>
>> 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 would still
>> need to
>> > > > update your proposal for people to comment and vote on.
>> > > >
>> > > >
>> > > > Guozhang
>> > > >
>> > > >
>> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com>
>> wrote:
>> > > >
>> > > > > +1
>> > > > >
>> > > > > One interface is cleaner.
>> > > > >
>> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
>> > interfaces
>> > > > > into 1
>> > > > > > interface.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Bill
>> > > > > >
>> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
>> > > je.karimov@gmail.com
>> > > > >
>> > > > > > 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 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
>> > > > > > this
>> > > > > > > would not be a problem.
>> > > > > > >
>> > > > > > > Our initial idea was similar to :
>> > > > > > >
>> > > > > > > public abstract class RichValueMapper<K, V, VR>  implements
>> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
>> > > > > > > ......
>> > > > > > > }
>> > > > > > >
>> > > > > > >
>> > > > > > > So, we check the type of object, whether it is RichXXX or
>> > > XXXWithKey
>> > > > > > inside
>> > > > > > > the called method and continue accordingly.
>> > > > > > >
>> > > > > > > If this is ok with the community, I would like to revert the
>> > > current
>> > > > > > design
>> > > > > > > to this again.
>> > > > > > >
>> > > > > > > Cheers,
>> > > > > > > Jeyhun
>> > > > > > >
>> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
>> damian.guy@gmail.com
>> > >
>> > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi Jeyhun,
>> > > > > > > >
>> > > > > > > > Thanks for sending out the update. I guess i was thinking
>> more
>> > > > along
>> > > > > > the
>> > > > > > > > lines of option 2 where we collapse the RichXXXX and
>> > > > ValueXXXXWithKey
>> > > > > > 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?
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Damian
>> > > > > > > >
>> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
>> > > je.karimov@gmail.com>
>> > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > 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 free to comment on this.
>> > > > > > > > >
>> > > > > > > > > [1]
>> > > > > > > > >
>> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
>> > > > > > > action?pageId=73637757
>> > > > > > > > >
>> > > > > > > > > Cheers,
>> > > > > > > > > Jeyhun
>> > > > > > > > >
>> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
>> > > > > je.karimov@gmail.com
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > 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 <
>> > > > > > > matthias@confluent.io>
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > >> 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
>> > > > > > > > > >> > for one-shot solution for overloads issue.
>> > > > > > > > > >> >
>> > > > > > > > > >> > Cheers,
>> > > > > > > > > >> > Jeyhun
>> > > > > > > > > >> >
>> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
>> > > > > > damian.guy@gmail.com>
>> > > > > > > > > >> wrote:
>> > > > > > > > > >> >
>> > > > > > > > > >> >> 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
>> > > > > > > > > overrides.
>> > > > > > > > > >> >>
>> > > > > > > > > >> >> Thanks,
>> > > > > > > > > >> >> Damian
>> > > > > > > > > >> >>
>> > > > > > > > > >> >>
>> > > > > > > > > >> >>> About ProcessorContext vs RecordContext, you are
>> > right.
>> > > I
>> > > > > > think
>> > > > > > > I
>> > > > > > > > > >> need to
>> > > > > > > > > >> >>> implement a prototype to understand the full
>> picture
>> > as
>> > > > some
>> > > > > > > parts
>> > > > > > > > > of
>> > > > > > > > > >> the
>> > > > > > > > > >> >>> KIP might not be as straightforward as I thought.
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>> Cheers,
>> > > > > > > > > >> >>> Jeyhun
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
>> > > > > > > damian.guy@gmail.com>
>> > > > > > > > > >> wrote:
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>>> 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
>> > > > > lifetime
>> > > > > > of
>> > > > > > > > the
>> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
>> > > > > > ProcessorContext
>> > > > > > > > to
>> > > > > > > > > a
>> > > > > > > > > >> >>>> RecordContext.
>> > > > > > > > > >> >>>> You mentioned above passing the
>> > > InternalProcessorContext
>> > > > to
>> > > > > > the
>> > > > > > > > > >> init()
>> > > > > > > > > >> >>>> calls. It is internal for a reason and i think it
>> > > should
>> > > > > > remain
>> > > > > > > > > that
>> > > > > > > > > >> >> way.
>> > > > > > > > > >> >>>> It might be better to move the recordContext()
>> method
>> > > > from
>> > > > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
>> > > > > > > > > >> >>>>
>> > > > > > > > > >> >>>> In the KIP you have an example showing:
>> > > > > > > > > >> >>>> richMapper.init((RecordContext) processorContext);
>> > > > > > > > > >> >>>> But the interface is:
>> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
>> > > > > > > > > >> >>>>     VR apply(final V value, final RecordContext
>> > > > > > recordContext);
>> > > > > > > > > >> >>>> }
>> > > > > > > > > >> >>>> i.e., there is no init(...), besides as above this
>> > > > wouldn't
>> > > > > > > make
>> > > > > > > > > >> sense.
>> > > > > > > > > >> >>>>
>> > > > > > > > > >> >>>> Thanks,
>> > > > > > > > > >> >>>> Damian
>> > > > > > > > > >> >>>>
>> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
>> > > > > > > je.karimov@gmail.com
>> > > > > > > > >
>> > > > > > > > > >> >> wrote:
>> > > > > > > > > >> >>>>
>> > > > > > > > > >> >>>>> 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.
>> Can't
>> > we
>> > > > pass
>> > > > > > > this
>> > > > > > > > > as a
>> > > > > > > > > >> >>>>> parameter to init() method of processors? Then we
>> > > would
>> > > > be
>> > > > > > > able
>> > > > > > > > to
>> > > > > > > > > >> >> get
>> > > > > > > > > >> >>>>> RecordContext easily with just a method call.
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>> Cheers,
>> > > > > > > > > >> >>>>> Jeyhun
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax
>> <
>> > > > > > > > > >> >>> matthias@confluent.io>
>> > > > > > > > > >> >>>>> wrote:
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>>> 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
>> > > > > > > > > >> >>> will
>> > > > > > > > > >> >>>>>> be provided by the consecutive call to aggregate
>> > > > anyways.
>> > > > > > > > > >> >>>>>>
>> > > > > > > > > >> >>>>>>
>> > > > > > > > > >> >>>>>> -Matthias
>> > > > > > > > > >> >>>>>>
>> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
>> > > > > > > > > >> >>>>>>> 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 implement
>> both.
>> > > This
>> > > > > > allows
>> > > > > > > > us
>> > > > > > > > > >> >> to
>> > > > > > > > > >> >>>> cast
>> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the
>> visible
>> > > > scope.
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>> -Matthias
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
>> > > > > > > > > >> >>>>>>>> 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 address this issue as well or
>> > > > > > > > > >> >>>>>>>> continue as it is?
>> > > > > > > > > >> >>>>>>>>
>> > > > > > > > > >> >>>>>>>> Cheers,
>> > > > > > > > > >> >>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>
>> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang
>> Wang <
>> > > > > > > > > >> >>> wangguoz@gmail.com>
>> > > > > > > > > >> >>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>
>> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>> Guozhang
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun
>> Karimov
>> > <
>> > > > > > > > > >> >>>>> je.karimov@gmail.com>
>> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can
>> > also
>> > > > be
>> > > > > > > > > >> >> integrated
>> > > > > > > > > >> >>>> into
>> > > > > > > > > >> >>>>>> the
>> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
>> > RecordContext
>> > > > > > > > interface.
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>> [1]
>> > > > https://issues.apache.org/jira/browse/KAFKA-3907
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>> Cheers,
>> > > > > > > > > >> >>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J.
>> > Sax <
>> > > > > > > > > >> >>>>>> matthias@confluent.io>
>> > > > > > > > > >> >>>>>>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> 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 information
>> > only.
>> > > > If
>> > > > > > you
>> > > > > > > > want
>> > > > > > > > > >> >> to
>> > > > > > > > > >> >>>> do
>> > > > > > > > > >> >>>>> a
>> > > > > > > > > >> >>>>>>>>>>> stateful computation you should rather use
>> > > > > > #transform().
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need
>> to
>> > > > switch
>> > > > > > to
>> > > > > > > a
>> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
>> > > overloads.
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
>> > > > interface
>> > > > > > > with a
>> > > > > > > > > >> >>> single
>> > > > > > > > > >> >>>>>>>>> method
>> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> -Matthias
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>> > > > > > > > > >> >>>>>>>>>>>> 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
>> > > <
>> > > > > > > > > >> >>>>>> je.karimov@gmail.com
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will not
>> mention
>> > > > them.
>> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in
>> > KIP-149
>> > > > and
>> > > > > in
>> > > > > > > > this
>> > > > > > > > > >> >> KIP
>> > > > > > > > > >> >>>>>>>>>> (KIP-159)
>> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into
>> the
>> > > > stage
>> > > > > > with
>> > > > > > > > > >> >>> KIP-149,
>> > > > > > > > > >> >>>>> in
>> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we
>> extended
>> > > > KIP-149
>> > > > > > to
>> > > > > > > > > >> >> support
>> > > > > > > > > >> >>>>> Rich
>> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we
>> > > > provided
>> > > > > > init
>> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
>> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested
>> that
>> > we
>> > > > > should
>> > > > > > > not
>> > > > > > > > > >> >>>> provide
>> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result,
>> we
>> > > > > separated
>> > > > > > > the
>> > > > > > > > > >> >> two
>> > > > > > > > > >> >>>>>>>>> problems
>> > > > > > > > > >> >>>>>>>>>>> into
>> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can
>> be
>> > > > solved
>> > > > > in
>> > > > > > > > > >> >>> parallel.
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V,
>> > VR>
>> > > {
>> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
>> > > > > > > > > >> >>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V,
>> VR>
>> > > > extends
>> > > > > > > > > >> >>>> RichFunction{
>> > > > > > > > > >> >>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
>> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
>> recordContext);
>> > > > > > > > > >> >>>>>>>>>>>>>     void close();
>> > > > > > > > > >> >>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
>> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
>> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
>> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
>> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
>> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
>> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
>> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
>> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
>> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
>> > > > > appConfigsWithPrefix(String
>> > > > > > > > > >> >> prefix);
>> > > > > > > > > >> >>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
>> > > > > > > RecordContext
>> > > > > > > > {
>> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
>> > > RecordContext
>> > > > > > > > > >> >>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> As a result:
>> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
>> > interfaces
>> > > > can
>> > > > > be
>> > > > > > > > > >> >>> converted
>> > > > > > > > > >> >>>> to
>> > > > > > > > > >> >>>>>>>>>> their
>> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and
>> > > close()
>> > > > > > > > methods)
>> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept
>> Rich
>> > > > > > interfaces
>> > > > > > > in
>> > > > > > > > > >> >>> their
>> > > > > > > > > >> >>>>>>>>>>>>> constructors.
>> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey"
>> or
>> > > > > > > "withoutKey"
>> > > > > > > > > >> >>>>> interfaces
>> > > > > > > > > >> >>>>>>>>> to
>> > > > > > > > > >> >>>>>>>>>>> Rich
>> > > > > > > > > >> >>>>>>>>>>>>> interface while building the topology and
>> > > > > initialize
>> > > > > > > the
>> > > > > > > > > >> >>>> related
>> > > > > > > > > >> >>>>>>>>>>> processors
>> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
>> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods
>> > for
>> > > > rich
>> > > > > > > > > >> >> functions
>> > > > > > > > > >> >>> as
>> > > > > > > > > >> >>>>>> Rich
>> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We
>> > will
>> > > > just
>> > > > > > > check
>> > > > > > > > > >> >> the
>> > > > > > > > > >> >>>>> object
>> > > > > > > > > >> >>>>>>>>>> type
>> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above
>> > > > approach
>> > > > > > > does
>> > > > > > > > > not
>> > > > > > > > > >> >>>>> support
>> > > > > > > > > >> >>>>>>>>>>> lambdas
>> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one method,
>> only
>> > > > > > > > > >> >>> init(RecordContext),
>> > > > > > > > > >> >>>>> as
>> > > > > > > > > >> >>>>>>>>>> part
>> > > > > > > > > >> >>>>>>>>>>> of
>> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
>> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I
>> > > think
>> > > > > Rich
>> > > > > > > > > >> >>> interfaces
>> > > > > > > > > >> >>>>> are
>> > > > > > > > > >> >>>>>>>>> by
>> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not
>> > care
>> > > > much
>> > > > > > > about
>> > > > > > > > > >> >> it.
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
>> > > > alternative
>> > > > > > we
>> > > > > > > > > >> >>>> considered
>> > > > > > > > > >> >>>>>> was
>> > > > > > > > > >> >>>>>>>>>> to
>> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
>> > parameter.
>> > > > > This
>> > > > > > > > might
>> > > > > > > > > >> >>> even
>> > > > > > > > > >> >>>>>>>>> allow
>> > > > > > > > > >> >>>>>>>>>> to
>> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
>> > > > > RichFunction
>> > > > > > as
>> > > > > > > > we
>> > > > > > > > > >> >>>>> preserve
>> > > > > > > > > >> >>>>>>>>> the
>> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
>> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
>> > > > > basically
>> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state
>> for a
>> > > > > > function.
>> > > > > > > > > >> >> Thus,
>> > > > > > > > > >> >>> we
>> > > > > > > > > >> >>>>>>>>> cannot
>> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of
>> RichValueMapper
>> > > (etc)
>> > > > > > over
>> > > > > > > > > >> >>> multiple
>> > > > > > > > > >> >>>>>> tasks
>> > > > > > > > > >> >>>>>>>>>> and
>> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar
>> to
>> > > > > > > > #transform().
>> > > > > > > > > >> >> And
>> > > > > > > > > >> >>>>> this
>> > > > > > > > > >> >>>>>>>>>> would
>> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
>> > > > > > > > (Rich)ValueMapperSupplier
>> > > > > > > > > >> >>> would
>> > > > > > > > > >> >>>>> not
>> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we
>> would
>> > > need
>> > > > > many
>> > > > > > > new
>> > > > > > > > > >> >>>> overload
>> > > > > > > > > >> >>>>>>>>> for
>> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
>> > > > > Matthias's
>> > > > > > > > > email)
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
>> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias
>> J.
>> > > Sax <
>> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>> 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
>> > > > 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:
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
>> > RecordContext
>> > > > > > > context);
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> }
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> ...
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
>> > mapValues(ValueMapper<?
>> > > > > super
>> > > > > > > V, ?
>> > > > > > > > > >> >>>> extends
>> > > > > > > > > >> >>>>>> VR>
>> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
>> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
>> > > > > > > > mapValueswithContext(RichValueMapper
>> > > > > > > > > >> >> <?
>> > > > > > > > > >> >>>>> super
>> > > > > > > > > >> >>>>>>>>>> V, ?
>> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces
>> more
>> > > > > > > overloads;
>> > > > > > > > > >> >> but I
>> > > > > > > > > >> >>>>> think
>> > > > > > > > > >> >>>>>>>>>> the
>> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1)
>> > > serde
>> > > > > > > > overrides
>> > > > > > > > > >> >> and
>> > > > > > > > > >> >>>> 2)
>> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of
>> > which
>> > > > can
>> > > > > > be
>> > > > > > > > > >> >> reduced
>> > > > > > > > > >> >>>> in
>> > > > > > > > > >> >>>>>> the
>> > > > > > > > > >> >>>>>>>>>>> near
>> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is
>> > still
>> > > > > > > > worthwhile,
>> > > > > > > > > >> >> as
>> > > > > > > > > >> >>>> it
>> > > > > > > > > >> >>>>>> has
>> > > > > > > > > >> >>>>>>>>>> the
>> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
>> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to
>> > > "convert"
>> > > > > > from
>> > > > > > > > > >> >>> non-rich
>> > > > > > > > > >> >>>>>>>>>> functions
>> > > > > > > > > >> >>>>>>>>>>>>> to
>> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been
>> > > discussed
>> > > > > > and I
>> > > > > > > > may
>> > > > > > > > > >> >>> have
>> > > > > > > > > >> >>>>>>>>>>>>> overlooked
>> > > > > > > > > >> >>>>>>>>>>>>>> in
>> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM,
>> Matthias
>> > J.
>> > > > > Sax <
>> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
>> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
>> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge
>> to
>> > an
>> > > > > > overall
>> > > > > > > > > >> >>>> consisted
>> > > > > > > > > >> >>>>>>>>> API.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we
>> > might
>> > > > get
>> > > > > > > more
>> > > > > > > > > >> >>>> overload.
>> > > > > > > > > >> >>>>>> It
>> > > > > > > > > >> >>>>>>>>>>>>> might
>> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
>> > > > > > > xxxWithContext()
>> > > > > > > > > >> >>>> overload
>> > > > > > > > > >> >>>>>>>>> that
>> > > > > > > > > >> >>>>>>>>>>>>> will
>> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise,
>> if
>> > > > might
>> > > > > > get
>> > > > > > > > too
>> > > > > > > > > >> >>> messy
>> > > > > > > > > >> >>>>>>>>> having
>> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
>> > > > > > > > > ValueMapperWithContext,
>> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the
>> > > "builder
>> > > > > > > pattern"
>> > > > > > > > > >> >> idea
>> > > > > > > > > >> >>>> as
>> > > > > > > > > >> >>>>> an
>> > > > > > > > > >> >>>>>>>>>> API
>> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the
>> > overload
>> > > > > > > problem.
>> > > > > > > > > Not
>> > > > > > > > > >> >>> for
>> > > > > > > > > >> >>>>>>>>> simple
>> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for
>> > joins
>> > > > and
>> > > > > > > > > >> >>>> aggregations.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in
>> an
>> > > older
>> > > > > > > email,
>> > > > > > > > I
>> > > > > > > > > >> >> am
>> > > > > > > > > >> >>>>>>>>>> personally
>> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
>> > > interface,
>> > > > > and
>> > > > > > > add
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with
>> > method
>> > > > > > > > > >> >>>>>> `open(RecordContext)`
>> > > > > > > > > >> >>>>>>>>>> (or
>> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) --
>> but
>> > not
>> > > > > > > > `close()`)
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
>> > ValueMapperWithRecordContext
>> > > > > > extends
>> > > > > > > > > >> >>>>> ValueMapper,
>> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
>> > overload.
>> > > Of
>> > > > > > > course,
>> > > > > > > > > we
>> > > > > > > > > >> >>>> don't
>> > > > > > > > > >> >>>>>>>>> get
>> > > > > > > > > >> >>>>>>>>>> a
>> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
>> > > sacrifices
>> > > > > > > > Lambdas.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided
>> > what
>> > > > the
>> > > > > > > > better
>> > > > > > > > > >> >>>> option
>> > > > > > > > > >> >>>>>>>>> might
>> > > > > > > > > >> >>>>>>>>>>>>> be.
>> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about
>> > this
>> > > > > trade
>> > > > > > > > off.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov
>> wrote:
>> > > > > > > > > >> >>>>>>>>>>>>>>>>> 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.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
>> > > > > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM
>> Guozhang
>> > > > Wang <
>> > > > > > > > > >> >>>>>>>>> wangguoz@gmail.com>
>> > > > > > > > > >> >>>>>>>>>>>>>> wrote:
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
>> > > > > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>
>> > > > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > Re+DISCUSS+KIP+149+Enabling+key+access+in+
>> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
>> > > > > functions+to+Streams
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> --
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>> --
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
>> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>>> --
>> > > > > > > > > >> >>>>>>>>>>>>> -Cheers
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>>> --
>> > > > > > > > > >> >>>>>>>>>> -Cheers
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>> Jeyhun
>> > > > > > > > > >> >>>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>>> --
>> > > > > > > > > >> >>>>>>>>> -- Guozhang
>> > > > > > > > > >> >>>>>>>>>
>> > > > > > > > > >> >>>>>>>
>> > > > > > > > > >> >>>>>>
>> > > > > > > > > >> >>>>>> --
>> > > > > > > > > >> >>>>> -Cheers
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>> Jeyhun
>> > > > > > > > > >> >>>>>
>> > > > > > > > > >> >>>>
>> > > > > > > > > >> >>> --
>> > > > > > > > > >> >>> -Cheers
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>> Jeyhun
>> > > > > > > > > >> >>>
>> > > > > > > > > >> >>
>> > > > > > > > > >>
>> > > > > > > > > >> --
>> > > > > > > > > > -Cheers
>> > > > > > > > > >
>> > > > > > > > > > Jeyhun
>> > > > > > > > > >
>> > > > > > > > > --
>> > > > > > > > > -Cheers
>> > > > > > > > >
>> > > > > > > > > Jeyhun
>> > > > > > > > >
>> > > > > > > >
>> > > > > > > --
>> > > > > > > -Cheers
>> > > > > > >
>> > > > > > > Jeyhun
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > > >
>> > > >
>> > > > --
>> > > > -- Guozhang
>> > > >
>> > >
>> >
>>
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 <da...@gmail.com> 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 Jeyhun Karimov <je...@gmail.com> wrote:
>
> > 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 merge this KIP into the upcoming release, as KIP-159 is not voted yet
> > (because we want both KIP-149 and KIP-159 to be as an "atomic" merge).
> So
> > I decided to wait until KIP-182 gets stable (there are some minor updates
> > AFAIK) and update the KIP accordingly. Please correct me if I am wrong
> or I
> > misunderstood.
> >
> > Cheers,
> > Jeyhun
> >
> >
> > On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com> wrote:
> >
> > > +1
> > >
> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com> 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 would still need
> to
> > > > update your proposal for people to comment and vote on.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com>
> wrote:
> > > >
> > > > > +1
> > > > >
> > > > > One interface is cleaner.
> > > > >
> > > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com>
> > > wrote:
> > > > >
> > > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
> > interfaces
> > > > > into 1
> > > > > > interface.
> > > > > >
> > > > > > Thanks,
> > > > > > Bill
> > > > > >
> > > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > > je.karimov@gmail.com
> > > > >
> > > > > > 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 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
> > > > > > this
> > > > > > > would not be a problem.
> > > > > > >
> > > > > > > Our initial idea was similar to :
> > > > > > >
> > > > > > > public abstract class RichValueMapper<K, V, VR>  implements
> > > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > > > ......
> > > > > > > }
> > > > > > >
> > > > > > >
> > > > > > > So, we check the type of object, whether it is RichXXX or
> > > XXXWithKey
> > > > > > inside
> > > > > > > the called method and continue accordingly.
> > > > > > >
> > > > > > > If this is ok with the community, I would like to revert the
> > > current
> > > > > > design
> > > > > > > to this again.
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Jeyhun
> > > > > > >
> > > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <
> damian.guy@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jeyhun,
> > > > > > > >
> > > > > > > > Thanks for sending out the update. I guess i was thinking
> more
> > > > along
> > > > > > the
> > > > > > > > lines of option 2 where we collapse the RichXXXX and
> > > > ValueXXXXWithKey
> > > > > > 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?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Damian
> > > > > > > >
> > > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > > je.karimov@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > 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 free to comment on this.
> > > > > > > > >
> > > > > > > > > [1]
> > > > > > > > >
> > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > > > > > action?pageId=73637757
> > > > > > > > >
> > > > > > > > > Cheers,
> > > > > > > > > Jeyhun
> > > > > > > > >
> > > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > > > je.karimov@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > 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 <
> > > > > > > matthias@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> 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
> > > > > > > > > >> > for one-shot solution for overloads issue.
> > > > > > > > > >> >
> > > > > > > > > >> > Cheers,
> > > > > > > > > >> > Jeyhun
> > > > > > > > > >> >
> > > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > > > > damian.guy@gmail.com>
> > > > > > > > > >> wrote:
> > > > > > > > > >> >
> > > > > > > > > >> >> 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
> > > > > > > > > overrides.
> > > > > > > > > >> >>
> > > > > > > > > >> >> Thanks,
> > > > > > > > > >> >> Damian
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >> >>> About ProcessorContext vs RecordContext, you are
> > right.
> > > I
> > > > > > think
> > > > > > > I
> > > > > > > > > >> need to
> > > > > > > > > >> >>> implement a prototype to understand the full picture
> > as
> > > > some
> > > > > > > parts
> > > > > > > > > of
> > > > > > > > > >> the
> > > > > > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > > > > > >> >>>
> > > > > > > > > >> >>>
> > > > > > > > > >> >>> Cheers,
> > > > > > > > > >> >>> Jeyhun
> > > > > > > > > >> >>>
> > > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > > > > > damian.guy@gmail.com>
> > > > > > > > > >> wrote:
> > > > > > > > > >> >>>
> > > > > > > > > >> >>>> 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
> > > > > lifetime
> > > > > > of
> > > > > > > > the
> > > > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > > > > > ProcessorContext
> > > > > > > > to
> > > > > > > > > a
> > > > > > > > > >> >>>> RecordContext.
> > > > > > > > > >> >>>> You mentioned above passing the
> > > InternalProcessorContext
> > > > to
> > > > > > the
> > > > > > > > > >> init()
> > > > > > > > > >> >>>> calls. It is internal for a reason and i think it
> > > should
> > > > > > remain
> > > > > > > > > that
> > > > > > > > > >> >> way.
> > > > > > > > > >> >>>> It might be better to move the recordContext()
> method
> > > > from
> > > > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>> In the KIP you have an example showing:
> > > > > > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > > > > > >> >>>> But the interface is:
> > > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > > > > > >> >>>>     VR apply(final V value, final RecordContext
> > > > > > recordContext);
> > > > > > > > > >> >>>> }
> > > > > > > > > >> >>>> i.e., there is no init(...), besides as above this
> > > > wouldn't
> > > > > > > make
> > > > > > > > > >> sense.
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>> Thanks,
> > > > > > > > > >> >>>> Damian
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > > > > > je.karimov@gmail.com
> > > > > > > > >
> > > > > > > > > >> >> wrote:
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>>> 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. Can't
> > we
> > > > pass
> > > > > > > this
> > > > > > > > > as a
> > > > > > > > > >> >>>>> parameter to init() method of processors? Then we
> > > would
> > > > be
> > > > > > > able
> > > > > > > > to
> > > > > > > > > >> >> get
> > > > > > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>> Cheers,
> > > > > > > > > >> >>>>> Jeyhun
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > > > > > >> >>> matthias@confluent.io>
> > > > > > > > > >> >>>>> wrote:
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>>> 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
> > > > > > > > > >> >>> will
> > > > > > > > > >> >>>>>> be provided by the consecutive call to aggregate
> > > > anyways.
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>>> -Matthias
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > > > > > >> >>>>>>> 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 implement both.
> > > This
> > > > > > allows
> > > > > > > > us
> > > > > > > > > >> >> to
> > > > > > > > > >> >>>> cast
> > > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible
> > > > scope.
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>> -Matthias
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > > > > > >> >>>>>>>> continue as it is?
> > > > > > > > > >> >>>>>>>>
> > > > > > > > > >> >>>>>>>> Cheers,
> > > > > > > > > >> >>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>
> > > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang
> <
> > > > > > > > > >> >>> wangguoz@gmail.com>
> > > > > > > > > >> >>>>>> wrote:
> > > > > > > > > >> >>>>>>>>
> > > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>> Guozhang
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun
> Karimov
> > <
> > > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can
> > also
> > > > be
> > > > > > > > > >> >> integrated
> > > > > > > > > >> >>>> into
> > > > > > > > > >> >>>>>> the
> > > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> > RecordContext
> > > > > > > > interface.
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>> [1]
> > > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>> Cheers,
> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J.
> > Sax <
> > > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > > > > >> >>>>>>>>>> wrote:
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> 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 information
> > only.
> > > > If
> > > > > > you
> > > > > > > > want
> > > > > > > > > >> >> to
> > > > > > > > > >> >>>> do
> > > > > > > > > >> >>>>> a
> > > > > > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > > > > > #transform().
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need
> to
> > > > switch
> > > > > > to
> > > > > > > a
> > > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
> > > overloads.
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
> > > > interface
> > > > > > > with a
> > > > > > > > > >> >>> single
> > > > > > > > > >> >>>>>>>>> method
> > > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > > > > > >> >>>>>>>>>>>> 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
> > > <
> > > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> wrote:
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> 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, so I will not
> mention
> > > > them.
> > > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in
> > KIP-149
> > > > and
> > > > > in
> > > > > > > > this
> > > > > > > > > >> >> KIP
> > > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into
> the
> > > > stage
> > > > > > with
> > > > > > > > > >> >>> KIP-149,
> > > > > > > > > >> >>>>> in
> > > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended
> > > > KIP-149
> > > > > > to
> > > > > > > > > >> >> support
> > > > > > > > > >> >>>>> Rich
> > > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we
> > > > provided
> > > > > > init
> > > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that
> > we
> > > > > should
> > > > > > > not
> > > > > > > > > >> >>>> provide
> > > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> > > > > separated
> > > > > > > the
> > > > > > > > > >> >> two
> > > > > > > > > >> >>>>>>>>> problems
> > > > > > > > > >> >>>>>>>>>>> into
> > > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be
> > > > solved
> > > > > in
> > > > > > > > > >> >>> parallel.
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V,
> > VR>
> > > {
> > > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR>
> > > > extends
> > > > > > > > > >> >>>> RichFunction{
> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext
> recordContext);
> > > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > > appConfigsWithPrefix(String
> > > > > > > > > >> >> prefix);
> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > > > > > RecordContext
> > > > > > > > {
> > > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> > > RecordContext
> > > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
> > interfaces
> > > > can
> > > > > be
> > > > > > > > > >> >>> converted
> > > > > > > > > >> >>>> to
> > > > > > > > > >> >>>>>>>>>> their
> > > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and
> > > close()
> > > > > > > > methods)
> > > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > > > > > interfaces
> > > > > > > in
> > > > > > > > > >> >>> their
> > > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > > > > > "withoutKey"
> > > > > > > > > >> >>>>> interfaces
> > > > > > > > > >> >>>>>>>>> to
> > > > > > > > > >> >>>>>>>>>>> Rich
> > > > > > > > > >> >>>>>>>>>>>>> interface while building the topology and
> > > > > initialize
> > > > > > > the
> > > > > > > > > >> >>>> related
> > > > > > > > > >> >>>>>>>>>>> processors
> > > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods
> > for
> > > > rich
> > > > > > > > > >> >> functions
> > > > > > > > > >> >>> as
> > > > > > > > > >> >>>>>> Rich
> > > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We
> > will
> > > > just
> > > > > > > check
> > > > > > > > > >> >> the
> > > > > > > > > >> >>>>> object
> > > > > > > > > >> >>>>>>>>>> type
> > > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above
> > > > approach
> > > > > > > does
> > > > > > > > > not
> > > > > > > > > >> >>>>> support
> > > > > > > > > >> >>>>>>>>>>> lambdas
> > > > > > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > > > > > >> >>> init(RecordContext),
> > > > > > > > > >> >>>>> as
> > > > > > > > > >> >>>>>>>>>> part
> > > > > > > > > >> >>>>>>>>>>> of
> > > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I
> > > think
> > > > > Rich
> > > > > > > > > >> >>> interfaces
> > > > > > > > > >> >>>>> are
> > > > > > > > > >> >>>>>>>>> by
> > > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not
> > care
> > > > much
> > > > > > > about
> > > > > > > > > >> >> it.
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
> > > > alternative
> > > > > > we
> > > > > > > > > >> >>>> considered
> > > > > > > > > >> >>>>>> was
> > > > > > > > > >> >>>>>>>>>> to
> > > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
> > parameter.
> > > > > This
> > > > > > > > might
> > > > > > > > > >> >>> even
> > > > > > > > > >> >>>>>>>>> allow
> > > > > > > > > >> >>>>>>>>>> to
> > > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> > > > > RichFunction
> > > > > > as
> > > > > > > > we
> > > > > > > > > >> >>>>> preserve
> > > > > > > > > >> >>>>>>>>> the
> > > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> > > > > basically
> > > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state
> for a
> > > > > > function.
> > > > > > > > > >> >> Thus,
> > > > > > > > > >> >>> we
> > > > > > > > > >> >>>>>>>>> cannot
> > > > > > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper
> > > (etc)
> > > > > > over
> > > > > > > > > >> >>> multiple
> > > > > > > > > >> >>>>>> tasks
> > > > > > > > > >> >>>>>>>>>> and
> > > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar
> to
> > > > > > > > #transform().
> > > > > > > > > >> >> And
> > > > > > > > > >> >>>>> this
> > > > > > > > > >> >>>>>>>>>> would
> > > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > > > > > (Rich)ValueMapperSupplier
> > > > > > > > > >> >>> would
> > > > > > > > > >> >>>>> not
> > > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would
> > > need
> > > > > many
> > > > > > > new
> > > > > > > > > >> >>>> overload
> > > > > > > > > >> >>>>>>>>> for
> > > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> > > > > Matthias's
> > > > > > > > > email)
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J.
> > > Sax <
> > > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>> 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
> > > > 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:
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> > RecordContext
> > > > > > > context);
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> }
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > mapValues(ValueMapper<?
> > > > > super
> > > > > > > V, ?
> > > > > > > > > >> >>>> extends
> > > > > > > > > >> >>>>>> VR>
> > > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > > > > mapValueswithContext(RichValueMapper
> > > > > > > > > >> >> <?
> > > > > > > > > >> >>>>> super
> > > > > > > > > >> >>>>>>>>>> V, ?
> > > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces
> more
> > > > > > > overloads;
> > > > > > > > > >> >> but I
> > > > > > > > > >> >>>>> think
> > > > > > > > > >> >>>>>>>>>> the
> > > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1)
> > > serde
> > > > > > > > overrides
> > > > > > > > > >> >> and
> > > > > > > > > >> >>>> 2)
> > > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of
> > which
> > > > can
> > > > > > be
> > > > > > > > > >> >> reduced
> > > > > > > > > >> >>>> in
> > > > > > > > > >> >>>>>> the
> > > > > > > > > >> >>>>>>>>>>> near
> > > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is
> > still
> > > > > > > > worthwhile,
> > > > > > > > > >> >> as
> > > > > > > > > >> >>>> it
> > > > > > > > > >> >>>>>> has
> > > > > > > > > >> >>>>>>>>>> the
> > > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to
> > > "convert"
> > > > > > from
> > > > > > > > > >> >>> non-rich
> > > > > > > > > >> >>>>>>>>>> functions
> > > > > > > > > >> >>>>>>>>>>>>> to
> > > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been
> > > discussed
> > > > > > and I
> > > > > > > > may
> > > > > > > > > >> >>> have
> > > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > > > > > >> >>>>>>>>>>>>>> in
> > > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM,
> Matthias
> > J.
> > > > > Sax <
> > > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge
> to
> > an
> > > > > > overall
> > > > > > > > > >> >>>> consisted
> > > > > > > > > >> >>>>>>>>> API.
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we
> > might
> > > > get
> > > > > > > more
> > > > > > > > > >> >>>> overload.
> > > > > > > > > >> >>>>>> It
> > > > > > > > > >> >>>>>>>>>>>>> might
> > > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > > > > > xxxWithContext()
> > > > > > > > > >> >>>> overload
> > > > > > > > > >> >>>>>>>>> that
> > > > > > > > > >> >>>>>>>>>>>>> will
> > > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise,
> if
> > > > might
> > > > > > get
> > > > > > > > too
> > > > > > > > > >> >>> messy
> > > > > > > > > >> >>>>>>>>> having
> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > > > > > ValueMapperWithContext,
> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the
> > > "builder
> > > > > > > pattern"
> > > > > > > > > >> >> idea
> > > > > > > > > >> >>>> as
> > > > > > > > > >> >>>>> an
> > > > > > > > > >> >>>>>>>>>> API
> > > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the
> > overload
> > > > > > > problem.
> > > > > > > > > Not
> > > > > > > > > >> >>> for
> > > > > > > > > >> >>>>>>>>> simple
> > > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for
> > joins
> > > > and
> > > > > > > > > >> >>>> aggregations.
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an
> > > older
> > > > > > > email,
> > > > > > > > I
> > > > > > > > > >> >> am
> > > > > > > > > >> >>>>>>>>>> personally
> > > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
> > > interface,
> > > > > and
> > > > > > > add
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with
> > method
> > > > > > > > > >> >>>>>> `open(RecordContext)`
> > > > > > > > > >> >>>>>>>>>> (or
> > > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but
> > not
> > > > > > > > `close()`)
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> > ValueMapperWithRecordContext
> > > > > > extends
> > > > > > > > > >> >>>>> ValueMapper,
> > > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
> > overload.
> > > Of
> > > > > > > course,
> > > > > > > > > we
> > > > > > > > > >> >>>> don't
> > > > > > > > > >> >>>>>>>>> get
> > > > > > > > > >> >>>>>>>>>> a
> > > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
> > > sacrifices
> > > > > > > > Lambdas.
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided
> > what
> > > > the
> > > > > > > > better
> > > > > > > > > >> >>>> option
> > > > > > > > > >> >>>>>>>>> might
> > > > > > > > > >> >>>>>>>>>>>>> be.
> > > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about
> > this
> > > > > trade
> > > > > > > > off.
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov
> wrote:
> > > > > > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM
> Guozhang
> > > > Wang <
> > > > > > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>
> > > > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> > > > > functions+to+Streams
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>>> --
> > > > > > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>> --
> > > > > > > > > >> >>>>>>>>>> -Cheers
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>>> --
> > > > > > > > > >> >>>>>>>>> -- Guozhang
> > > > > > > > > >> >>>>>>>>>
> > > > > > > > > >> >>>>>>>
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>>> --
> > > > > > > > > >> >>>>> -Cheers
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>> Jeyhun
> > > > > > > > > >> >>>>>
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>> --
> > > > > > > > > >> >>> -Cheers
> > > > > > > > > >> >>>
> > > > > > > > > >> >>> Jeyhun
> > > > > > > > > >> >>>
> > > > > > > > > >> >>
> > > > > > > > > >>
> > > > > > > > > >> --
> > > > > > > > > > -Cheers
> > > > > > > > > >
> > > > > > > > > > Jeyhun
> > > > > > > > > >
> > > > > > > > > --
> > > > > > > > > -Cheers
> > > > > > > > >
> > > > > > > > > Jeyhun
> > > > > > > > >
> > > > > > > >
> > > > > > > --
> > > > > > > -Cheers
> > > > > > >
> > > > > > > Jeyhun
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

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

Posted by Damian Guy <da...@gmail.com>.
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 <je...@gmail.com> wrote:

> 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 merge this KIP into the upcoming release, as KIP-159 is not voted yet
> (because we want both KIP-149 and KIP-159 to be as an "atomic" merge).  So
> I decided to wait until KIP-182 gets stable (there are some minor updates
> AFAIK) and update the KIP accordingly. Please correct me if I am wrong or I
> misunderstood.
>
> Cheers,
> Jeyhun
>
>
> On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com> wrote:
>
> > +1
> >
> > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com> 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 would still need to
> > > update your proposal for people to comment and vote on.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com> wrote:
> > >
> > > > +1
> > > >
> > > > One interface is cleaner.
> > > >
> > > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com>
> > wrote:
> > > >
> > > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey
> interfaces
> > > > into 1
> > > > > interface.
> > > > >
> > > > > Thanks,
> > > > > Bill
> > > > >
> > > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> > je.karimov@gmail.com
> > > >
> > > > > 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 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
> > > > > this
> > > > > > would not be a problem.
> > > > > >
> > > > > > Our initial idea was similar to :
> > > > > >
> > > > > > public abstract class RichValueMapper<K, V, VR>  implements
> > > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > > ......
> > > > > > }
> > > > > >
> > > > > >
> > > > > > So, we check the type of object, whether it is RichXXX or
> > XXXWithKey
> > > > > inside
> > > > > > the called method and continue accordingly.
> > > > > >
> > > > > > If this is ok with the community, I would like to revert the
> > current
> > > > > design
> > > > > > to this again.
> > > > > >
> > > > > > Cheers,
> > > > > > Jeyhun
> > > > > >
> > > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <damian.guy@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Hi Jeyhun,
> > > > > > >
> > > > > > > Thanks for sending out the update. I guess i was thinking more
> > > along
> > > > > the
> > > > > > > lines of option 2 where we collapse the RichXXXX and
> > > ValueXXXXWithKey
> > > > > 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?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Damian
> > > > > > >
> > > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> > je.karimov@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > 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 free to comment on this.
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > > > > action?pageId=73637757
> > > > > > > >
> > > > > > > > Cheers,
> > > > > > > > Jeyhun
> > > > > > > >
> > > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > > je.karimov@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > 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 <
> > > > > > matthias@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> 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
> > > > > > > > >> > for one-shot solution for overloads issue.
> > > > > > > > >> >
> > > > > > > > >> > Cheers,
> > > > > > > > >> > Jeyhun
> > > > > > > > >> >
> > > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > > > damian.guy@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >
> > > > > > > > >> >> 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
> > > > > > > > overrides.
> > > > > > > > >> >>
> > > > > > > > >> >> Thanks,
> > > > > > > > >> >> Damian
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>> About ProcessorContext vs RecordContext, you are
> right.
> > I
> > > > > think
> > > > > > I
> > > > > > > > >> need to
> > > > > > > > >> >>> implement a prototype to understand the full picture
> as
> > > some
> > > > > > parts
> > > > > > > > of
> > > > > > > > >> the
> > > > > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > > > > >> >>>
> > > > > > > > >> >>>
> > > > > > > > >> >>> Cheers,
> > > > > > > > >> >>> Jeyhun
> > > > > > > > >> >>>
> > > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > > > > damian.guy@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >>>
> > > > > > > > >> >>>> 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
> > > > lifetime
> > > > > of
> > > > > > > the
> > > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > > > > ProcessorContext
> > > > > > > to
> > > > > > > > a
> > > > > > > > >> >>>> RecordContext.
> > > > > > > > >> >>>> You mentioned above passing the
> > InternalProcessorContext
> > > to
> > > > > the
> > > > > > > > >> init()
> > > > > > > > >> >>>> calls. It is internal for a reason and i think it
> > should
> > > > > remain
> > > > > > > > that
> > > > > > > > >> >> way.
> > > > > > > > >> >>>> It might be better to move the recordContext() method
> > > from
> > > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> In the KIP you have an example showing:
> > > > > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > > > > >> >>>> But the interface is:
> > > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > > > > >> >>>>     VR apply(final V value, final RecordContext
> > > > > recordContext);
> > > > > > > > >> >>>> }
> > > > > > > > >> >>>> i.e., there is no init(...), besides as above this
> > > wouldn't
> > > > > > make
> > > > > > > > >> sense.
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> Thanks,
> > > > > > > > >> >>>> Damian
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > > > > je.karimov@gmail.com
> > > > > > > >
> > > > > > > > >> >> wrote:
> > > > > > > > >> >>>>
> > > > > > > > >> >>>>> 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. Can't
> we
> > > pass
> > > > > > this
> > > > > > > > as a
> > > > > > > > >> >>>>> parameter to init() method of processors? Then we
> > would
> > > be
> > > > > > able
> > > > > > > to
> > > > > > > > >> >> get
> > > > > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> Cheers,
> > > > > > > > >> >>>>> Jeyhun
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > > > > >> >>> matthias@confluent.io>
> > > > > > > > >> >>>>> wrote:
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>>> 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
> > > > > > > > >> >>> will
> > > > > > > > >> >>>>>> be provided by the consecutive call to aggregate
> > > anyways.
> > > > > > > > >> >>>>>>
> > > > > > > > >> >>>>>>
> > > > > > > > >> >>>>>> -Matthias
> > > > > > > > >> >>>>>>
> > > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > > > > >> >>>>>>> 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 implement both.
> > This
> > > > > allows
> > > > > > > us
> > > > > > > > >> >> to
> > > > > > > > >> >>>> cast
> > > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible
> > > scope.
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>> -Matthias
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > > > > >> >>>>>>>> continue as it is?
> > > > > > > > >> >>>>>>>>
> > > > > > > > >> >>>>>>>> Cheers,
> > > > > > > > >> >>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>
> > > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > > > > > >> >>> wangguoz@gmail.com>
> > > > > > > > >> >>>>>> wrote:
> > > > > > > > >> >>>>>>>>
> > > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>> Guozhang
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov
> <
> > > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can
> also
> > > be
> > > > > > > > >> >> integrated
> > > > > > > > >> >>>> into
> > > > > > > > >> >>>>>> the
> > > > > > > > >> >>>>>>>>>> KIP by adding related method inside
> RecordContext
> > > > > > > interface.
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>> [1]
> > > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>> Cheers,
> > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J.
> Sax <
> > > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > > > >> >>>>>>>>>> wrote:
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> 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 information
> only.
> > > If
> > > > > you
> > > > > > > want
> > > > > > > > >> >> to
> > > > > > > > >> >>>> do
> > > > > > > > >> >>>>> a
> > > > > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > > > > #transform().
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to
> > > switch
> > > > > to
> > > > > > a
> > > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
> > overloads.
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
> > > interface
> > > > > > with a
> > > > > > > > >> >>> single
> > > > > > > > >> >>>>>>>>> method
> > > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > > > > >> >>>>>>>>>>>> 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
> > <
> > > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> wrote:
> > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> 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, so I will not mention
> > > them.
> > > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in
> KIP-149
> > > and
> > > > in
> > > > > > > this
> > > > > > > > >> >> KIP
> > > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the
> > > stage
> > > > > with
> > > > > > > > >> >>> KIP-149,
> > > > > > > > >> >>>>> in
> > > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended
> > > KIP-149
> > > > > to
> > > > > > > > >> >> support
> > > > > > > > >> >>>>> Rich
> > > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we
> > > provided
> > > > > init
> > > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that
> we
> > > > should
> > > > > > not
> > > > > > > > >> >>>> provide
> > > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> > > > separated
> > > > > > the
> > > > > > > > >> >> two
> > > > > > > > >> >>>>>>>>> problems
> > > > > > > > >> >>>>>>>>>>> into
> > > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be
> > > solved
> > > > in
> > > > > > > > >> >>> parallel.
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V,
> VR>
> > {
> > > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR>
> > > extends
> > > > > > > > >> >>>> RichFunction{
> > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > > appConfigsWithPrefix(String
> > > > > > > > >> >> prefix);
> > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > > > > RecordContext
> > > > > > > {
> > > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> > RecordContext
> > > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey"
> interfaces
> > > can
> > > > be
> > > > > > > > >> >>> converted
> > > > > > > > >> >>>> to
> > > > > > > > >> >>>>>>>>>> their
> > > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and
> > close()
> > > > > > > methods)
> > > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > > > > interfaces
> > > > > > in
> > > > > > > > >> >>> their
> > > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > > > > "withoutKey"
> > > > > > > > >> >>>>> interfaces
> > > > > > > > >> >>>>>>>>> to
> > > > > > > > >> >>>>>>>>>>> Rich
> > > > > > > > >> >>>>>>>>>>>>> interface while building the topology and
> > > > initialize
> > > > > > the
> > > > > > > > >> >>>> related
> > > > > > > > >> >>>>>>>>>>> processors
> > > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods
> for
> > > rich
> > > > > > > > >> >> functions
> > > > > > > > >> >>> as
> > > > > > > > >> >>>>>> Rich
> > > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We
> will
> > > just
> > > > > > check
> > > > > > > > >> >> the
> > > > > > > > >> >>>>> object
> > > > > > > > >> >>>>>>>>>> type
> > > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above
> > > approach
> > > > > > does
> > > > > > > > not
> > > > > > > > >> >>>>> support
> > > > > > > > >> >>>>>>>>>>> lambdas
> > > > > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > > > > >> >>> init(RecordContext),
> > > > > > > > >> >>>>> as
> > > > > > > > >> >>>>>>>>>> part
> > > > > > > > >> >>>>>>>>>>> of
> > > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I
> > think
> > > > Rich
> > > > > > > > >> >>> interfaces
> > > > > > > > >> >>>>> are
> > > > > > > > >> >>>>>>>>> by
> > > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not
> care
> > > much
> > > > > > about
> > > > > > > > >> >> it.
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
> > > alternative
> > > > > we
> > > > > > > > >> >>>> considered
> > > > > > > > >> >>>>>> was
> > > > > > > > >> >>>>>>>>>> to
> > > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method
> parameter.
> > > > This
> > > > > > > might
> > > > > > > > >> >>> even
> > > > > > > > >> >>>>>>>>> allow
> > > > > > > > >> >>>>>>>>>> to
> > > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> > > > RichFunction
> > > > > as
> > > > > > > we
> > > > > > > > >> >>>>> preserve
> > > > > > > > >> >>>>>>>>> the
> > > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> > > > basically
> > > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> > > > > function.
> > > > > > > > >> >> Thus,
> > > > > > > > >> >>> we
> > > > > > > > >> >>>>>>>>> cannot
> > > > > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper
> > (etc)
> > > > > over
> > > > > > > > >> >>> multiple
> > > > > > > > >> >>>>>> tasks
> > > > > > > > >> >>>>>>>>>> and
> > > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > > > > > #transform().
> > > > > > > > >> >> And
> > > > > > > > >> >>>>> this
> > > > > > > > >> >>>>>>>>>> would
> > > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > > > > (Rich)ValueMapperSupplier
> > > > > > > > >> >>> would
> > > > > > > > >> >>>>> not
> > > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would
> > need
> > > > many
> > > > > > new
> > > > > > > > >> >>>> overload
> > > > > > > > >> >>>>>>>>> for
> > > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> > > > Matthias's
> > > > > > > > email)
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J.
> > Sax <
> > > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>> 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
> > > 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:
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final
> RecordContext
> > > > > > context);
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> }
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> mapValues(ValueMapper<?
> > > > super
> > > > > > V, ?
> > > > > > > > >> >>>> extends
> > > > > > > > >> >>>>>> VR>
> > > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > > > mapValueswithContext(RichValueMapper
> > > > > > > > >> >> <?
> > > > > > > > >> >>>>> super
> > > > > > > > >> >>>>>>>>>> V, ?
> > > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > > > > > overloads;
> > > > > > > > >> >> but I
> > > > > > > > >> >>>>> think
> > > > > > > > >> >>>>>>>>>> the
> > > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1)
> > serde
> > > > > > > overrides
> > > > > > > > >> >> and
> > > > > > > > >> >>>> 2)
> > > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of
> which
> > > can
> > > > > be
> > > > > > > > >> >> reduced
> > > > > > > > >> >>>> in
> > > > > > > > >> >>>>>> the
> > > > > > > > >> >>>>>>>>>>> near
> > > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is
> still
> > > > > > > worthwhile,
> > > > > > > > >> >> as
> > > > > > > > >> >>>> it
> > > > > > > > >> >>>>>> has
> > > > > > > > >> >>>>>>>>>> the
> > > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to
> > "convert"
> > > > > from
> > > > > > > > >> >>> non-rich
> > > > > > > > >> >>>>>>>>>> functions
> > > > > > > > >> >>>>>>>>>>>>> to
> > > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been
> > discussed
> > > > > and I
> > > > > > > may
> > > > > > > > >> >>> have
> > > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > > > > >> >>>>>>>>>>>>>> in
> > > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias
> J.
> > > > Sax <
> > > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to
> an
> > > > > overall
> > > > > > > > >> >>>> consisted
> > > > > > > > >> >>>>>>>>> API.
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we
> might
> > > get
> > > > > > more
> > > > > > > > >> >>>> overload.
> > > > > > > > >> >>>>>> It
> > > > > > > > >> >>>>>>>>>>>>> might
> > > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > > > > xxxWithContext()
> > > > > > > > >> >>>> overload
> > > > > > > > >> >>>>>>>>> that
> > > > > > > > >> >>>>>>>>>>>>> will
> > > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if
> > > might
> > > > > get
> > > > > > > too
> > > > > > > > >> >>> messy
> > > > > > > > >> >>>>>>>>> having
> > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > > > > ValueMapperWithContext,
> > > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the
> > "builder
> > > > > > pattern"
> > > > > > > > >> >> idea
> > > > > > > > >> >>>> as
> > > > > > > > >> >>>>> an
> > > > > > > > >> >>>>>>>>>> API
> > > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the
> overload
> > > > > > problem.
> > > > > > > > Not
> > > > > > > > >> >>> for
> > > > > > > > >> >>>>>>>>> simple
> > > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for
> joins
> > > and
> > > > > > > > >> >>>> aggregations.
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an
> > older
> > > > > > email,
> > > > > > > I
> > > > > > > > >> >> am
> > > > > > > > >> >>>>>>>>>> personally
> > > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
> > interface,
> > > > and
> > > > > > add
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with
> method
> > > > > > > > >> >>>>>> `open(RecordContext)`
> > > > > > > > >> >>>>>>>>>> (or
> > > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but
> not
> > > > > > > `close()`)
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>   - interface
> ValueMapperWithRecordContext
> > > > > extends
> > > > > > > > >> >>>>> ValueMapper,
> > > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any
> overload.
> > Of
> > > > > > course,
> > > > > > > > we
> > > > > > > > >> >>>> don't
> > > > > > > > >> >>>>>>>>> get
> > > > > > > > >> >>>>>>>>>> a
> > > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
> > sacrifices
> > > > > > > Lambdas.
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided
> what
> > > the
> > > > > > > better
> > > > > > > > >> >>>> option
> > > > > > > > >> >>>>>>>>> might
> > > > > > > > >> >>>>>>>>>>>>> be.
> > > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about
> this
> > > > trade
> > > > > > > off.
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang
> > > Wang <
> > > > > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>
> > > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> > > > functions+to+Streams
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>>> --
> > > > > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>> --
> > > > > > > > >> >>>>>>>>>> -Cheers
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>>> --
> > > > > > > > >> >>>>>>>>> -- Guozhang
> > > > > > > > >> >>>>>>>>>
> > > > > > > > >> >>>>>>>
> > > > > > > > >> >>>>>>
> > > > > > > > >> >>>>>> --
> > > > > > > > >> >>>>> -Cheers
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> Jeyhun
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>
> > > > > > > > >> >>> --
> > > > > > > > >> >>> -Cheers
> > > > > > > > >> >>>
> > > > > > > > >> >>> Jeyhun
> > > > > > > > >> >>>
> > > > > > > > >> >>
> > > > > > > > >>
> > > > > > > > >> --
> > > > > > > > > -Cheers
> > > > > > > > >
> > > > > > > > > Jeyhun
> > > > > > > > >
> > > > > > > > --
> > > > > > > > -Cheers
> > > > > > > >
> > > > > > > > Jeyhun
> > > > > > > >
> > > > > > >
> > > > > > --
> > > > > > -Cheers
> > > > > >
> > > > > > Jeyhun
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 merge this KIP into the upcoming release, as KIP-159 is not voted yet
(because we want both KIP-149 and KIP-159 to be as an "atomic" merge).  So
I decided to wait until KIP-182 gets stable (there are some minor updates
AFAIK) and update the KIP accordingly. Please correct me if I am wrong or I
misunderstood.

Cheers,
Jeyhun


On Thu, Sep 21, 2017 at 4:11 PM Damian Guy <da...@gmail.com> wrote:

> +1
>
> On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com> 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 would still need to
> > update your proposal for people to comment and vote on.
> >
> >
> > Guozhang
> >
> >
> > On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> > > +1
> > >
> > > One interface is cleaner.
> > >
> > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com>
> wrote:
> > >
> > > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces
> > > into 1
> > > > interface.
> > > >
> > > > Thanks,
> > > > Bill
> > > >
> > > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > > > 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 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
> > > > this
> > > > > would not be a problem.
> > > > >
> > > > > Our initial idea was similar to :
> > > > >
> > > > > public abstract class RichValueMapper<K, V, VR>  implements
> > > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > > ......
> > > > > }
> > > > >
> > > > >
> > > > > So, we check the type of object, whether it is RichXXX or
> XXXWithKey
> > > > inside
> > > > > the called method and continue accordingly.
> > > > >
> > > > > If this is ok with the community, I would like to revert the
> current
> > > > design
> > > > > to this again.
> > > > >
> > > > > Cheers,
> > > > > Jeyhun
> > > > >
> > > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hi Jeyhun,
> > > > > >
> > > > > > Thanks for sending out the update. I guess i was thinking more
> > along
> > > > the
> > > > > > lines of option 2 where we collapse the RichXXXX and
> > ValueXXXXWithKey
> > > > 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?
> > > > > >
> > > > > > Thanks,
> > > > > > Damian
> > > > > >
> > > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <
> je.karimov@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > 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 free to comment on this.
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > > > action?pageId=73637757
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Jeyhun
> > > > > > >
> > > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > > je.karimov@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > 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 <
> > > > > matthias@confluent.io>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > >> 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
> > > > > > > >> > for one-shot solution for overloads issue.
> > > > > > > >> >
> > > > > > > >> > Cheers,
> > > > > > > >> > Jeyhun
> > > > > > > >> >
> > > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > > damian.guy@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >
> > > > > > > >> >> 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
> > > > > > > overrides.
> > > > > > > >> >>
> > > > > > > >> >> Thanks,
> > > > > > > >> >> Damian
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>> About ProcessorContext vs RecordContext, you are right.
> I
> > > > think
> > > > > I
> > > > > > > >> need to
> > > > > > > >> >>> implement a prototype to understand the full picture as
> > some
> > > > > parts
> > > > > > > of
> > > > > > > >> the
> > > > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > > > >> >>>
> > > > > > > >> >>>
> > > > > > > >> >>> Cheers,
> > > > > > > >> >>> Jeyhun
> > > > > > > >> >>>
> > > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > > > damian.guy@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >>>
> > > > > > > >> >>>> 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
> > > lifetime
> > > > of
> > > > > > the
> > > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > > > ProcessorContext
> > > > > > to
> > > > > > > a
> > > > > > > >> >>>> RecordContext.
> > > > > > > >> >>>> You mentioned above passing the
> InternalProcessorContext
> > to
> > > > the
> > > > > > > >> init()
> > > > > > > >> >>>> calls. It is internal for a reason and i think it
> should
> > > > remain
> > > > > > > that
> > > > > > > >> >> way.
> > > > > > > >> >>>> It might be better to move the recordContext() method
> > from
> > > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > > > >> >>>>
> > > > > > > >> >>>> In the KIP you have an example showing:
> > > > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > > > >> >>>> But the interface is:
> > > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > > > >> >>>>     VR apply(final V value, final RecordContext
> > > > recordContext);
> > > > > > > >> >>>> }
> > > > > > > >> >>>> i.e., there is no init(...), besides as above this
> > wouldn't
> > > > > make
> > > > > > > >> sense.
> > > > > > > >> >>>>
> > > > > > > >> >>>> Thanks,
> > > > > > > >> >>>> Damian
> > > > > > > >> >>>>
> > > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > > > je.karimov@gmail.com
> > > > > > >
> > > > > > > >> >> wrote:
> > > > > > > >> >>>>
> > > > > > > >> >>>>> 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. Can't we
> > pass
> > > > > this
> > > > > > > as a
> > > > > > > >> >>>>> parameter to init() method of processors? Then we
> would
> > be
> > > > > able
> > > > > > to
> > > > > > > >> >> get
> > > > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > > > >> >>>>>
> > > > > > > >> >>>>>
> > > > > > > >> >>>>> Cheers,
> > > > > > > >> >>>>> Jeyhun
> > > > > > > >> >>>>>
> > > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > > > >> >>> matthias@confluent.io>
> > > > > > > >> >>>>> wrote:
> > > > > > > >> >>>>>
> > > > > > > >> >>>>>> 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
> > > > > > > >> >>> will
> > > > > > > >> >>>>>> be provided by the consecutive call to aggregate
> > anyways.
> > > > > > > >> >>>>>>
> > > > > > > >> >>>>>>
> > > > > > > >> >>>>>> -Matthias
> > > > > > > >> >>>>>>
> > > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > > > >> >>>>>>> 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 implement both.
> This
> > > > allows
> > > > > > us
> > > > > > > >> >> to
> > > > > > > >> >>>> cast
> > > > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible
> > scope.
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>> -Matthias
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > > > >> >>>>>>>> continue as it is?
> > > > > > > >> >>>>>>>>
> > > > > > > >> >>>>>>>> Cheers,
> > > > > > > >> >>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>
> > > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > > > > >> >>> wangguoz@gmail.com>
> > > > > > > >> >>>>>> wrote:
> > > > > > > >> >>>>>>>>
> > > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>> Guozhang
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also
> > be
> > > > > > > >> >> integrated
> > > > > > > >> >>>> into
> > > > > > > >> >>>>>> the
> > > > > > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > > > > > interface.
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>> [1]
> > https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>> Cheers,
> > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > > >> >>>>>>>>>> wrote:
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> 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 information only.
> > If
> > > > you
> > > > > > want
> > > > > > > >> >> to
> > > > > > > >> >>>> do
> > > > > > > >> >>>>> a
> > > > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > > > #transform().
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to
> > switch
> > > > to
> > > > > a
> > > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more
> overloads.
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
> > interface
> > > > > with a
> > > > > > > >> >>> single
> > > > > > > >> >>>>>>>>> method
> > > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > > > >> >>>>>>>>>>>> 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
> <
> > > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> wrote:
> > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> 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, so I will not mention
> > them.
> > > > > > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149
> > and
> > > in
> > > > > > this
> > > > > > > >> >> KIP
> > > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the
> > stage
> > > > with
> > > > > > > >> >>> KIP-149,
> > > > > > > >> >>>>> in
> > > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended
> > KIP-149
> > > > to
> > > > > > > >> >> support
> > > > > > > >> >>>>> Rich
> > > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we
> > provided
> > > > init
> > > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we
> > > should
> > > > > not
> > > > > > > >> >>>> provide
> > > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> > > separated
> > > > > the
> > > > > > > >> >> two
> > > > > > > >> >>>>>>>>> problems
> > > > > > > >> >>>>>>>>>>> into
> > > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be
> > solved
> > > in
> > > > > > > >> >>> parallel.
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR>
> {
> > > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR>
> > extends
> > > > > > > >> >>>> RichFunction{
> > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > > appConfigsWithPrefix(String
> > > > > > > >> >> prefix);
> > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > > > RecordContext
> > > > > > {
> > > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in
> RecordContext
> > > > > > > >> >>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces
> > can
> > > be
> > > > > > > >> >>> converted
> > > > > > > >> >>>> to
> > > > > > > >> >>>>>>>>>> their
> > > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and
> close()
> > > > > > methods)
> > > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > > > interfaces
> > > > > in
> > > > > > > >> >>> their
> > > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > > > "withoutKey"
> > > > > > > >> >>>>> interfaces
> > > > > > > >> >>>>>>>>> to
> > > > > > > >> >>>>>>>>>>> Rich
> > > > > > > >> >>>>>>>>>>>>> interface while building the topology and
> > > initialize
> > > > > the
> > > > > > > >> >>>> related
> > > > > > > >> >>>>>>>>>>> processors
> > > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for
> > rich
> > > > > > > >> >> functions
> > > > > > > >> >>> as
> > > > > > > >> >>>>>> Rich
> > > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will
> > just
> > > > > check
> > > > > > > >> >> the
> > > > > > > >> >>>>> object
> > > > > > > >> >>>>>>>>>> type
> > > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above
> > approach
> > > > > does
> > > > > > > not
> > > > > > > >> >>>>> support
> > > > > > > >> >>>>>>>>>>> lambdas
> > > > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > > > >> >>> init(RecordContext),
> > > > > > > >> >>>>> as
> > > > > > > >> >>>>>>>>>> part
> > > > > > > >> >>>>>>>>>>> of
> > > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I
> think
> > > Rich
> > > > > > > >> >>> interfaces
> > > > > > > >> >>>>> are
> > > > > > > >> >>>>>>>>> by
> > > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not care
> > much
> > > > > about
> > > > > > > >> >> it.
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
> > alternative
> > > > we
> > > > > > > >> >>>> considered
> > > > > > > >> >>>>>> was
> > > > > > > >> >>>>>>>>>> to
> > > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.
> > > This
> > > > > > might
> > > > > > > >> >>> even
> > > > > > > >> >>>>>>>>> allow
> > > > > > > >> >>>>>>>>>> to
> > > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> > > RichFunction
> > > > as
> > > > > > we
> > > > > > > >> >>>>> preserve
> > > > > > > >> >>>>>>>>> the
> > > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> > > basically
> > > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> > > > function.
> > > > > > > >> >> Thus,
> > > > > > > >> >>> we
> > > > > > > >> >>>>>>>>> cannot
> > > > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper
> (etc)
> > > > over
> > > > > > > >> >>> multiple
> > > > > > > >> >>>>>> tasks
> > > > > > > >> >>>>>>>>>> and
> > > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > > > > #transform().
> > > > > > > >> >> And
> > > > > > > >> >>>>> this
> > > > > > > >> >>>>>>>>>> would
> > > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > > > (Rich)ValueMapperSupplier
> > > > > > > >> >>> would
> > > > > > > >> >>>>> not
> > > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would
> need
> > > many
> > > > > new
> > > > > > > >> >>>> overload
> > > > > > > >> >>>>>>>>> for
> > > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> > > Matthias's
> > > > > > > email)
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J.
> Sax <
> > > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>> 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
> > 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:
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> > > > > context);
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> }
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<?
> > > super
> > > > > V, ?
> > > > > > > >> >>>> extends
> > > > > > > >> >>>>>> VR>
> > > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > > mapValueswithContext(RichValueMapper
> > > > > > > >> >> <?
> > > > > > > >> >>>>> super
> > > > > > > >> >>>>>>>>>> V, ?
> > > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > > > > overloads;
> > > > > > > >> >> but I
> > > > > > > >> >>>>> think
> > > > > > > >> >>>>>>>>>> the
> > > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1)
> serde
> > > > > > overrides
> > > > > > > >> >> and
> > > > > > > >> >>>> 2)
> > > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which
> > can
> > > > be
> > > > > > > >> >> reduced
> > > > > > > >> >>>> in
> > > > > > > >> >>>>>> the
> > > > > > > >> >>>>>>>>>>> near
> > > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > > > > > worthwhile,
> > > > > > > >> >> as
> > > > > > > >> >>>> it
> > > > > > > >> >>>>>> has
> > > > > > > >> >>>>>>>>>> the
> > > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to
> "convert"
> > > > from
> > > > > > > >> >>> non-rich
> > > > > > > >> >>>>>>>>>> functions
> > > > > > > >> >>>>>>>>>>>>> to
> > > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been
> discussed
> > > > and I
> > > > > > may
> > > > > > > >> >>> have
> > > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > > > >> >>>>>>>>>>>>>> in
> > > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J.
> > > Sax <
> > > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an
> > > > overall
> > > > > > > >> >>>> consisted
> > > > > > > >> >>>>>>>>> API.
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might
> > get
> > > > > more
> > > > > > > >> >>>> overload.
> > > > > > > >> >>>>>> It
> > > > > > > >> >>>>>>>>>>>>> might
> > > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > > > xxxWithContext()
> > > > > > > >> >>>> overload
> > > > > > > >> >>>>>>>>> that
> > > > > > > >> >>>>>>>>>>>>> will
> > > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if
> > might
> > > > get
> > > > > > too
> > > > > > > >> >>> messy
> > > > > > > >> >>>>>>>>> having
> > > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > > > ValueMapperWithContext,
> > > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the
> "builder
> > > > > pattern"
> > > > > > > >> >> idea
> > > > > > > >> >>>> as
> > > > > > > >> >>>>> an
> > > > > > > >> >>>>>>>>>> API
> > > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> > > > > problem.
> > > > > > > Not
> > > > > > > >> >>> for
> > > > > > > >> >>>>>>>>> simple
> > > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins
> > and
> > > > > > > >> >>>> aggregations.
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an
> older
> > > > > email,
> > > > > > I
> > > > > > > >> >> am
> > > > > > > >> >>>>>>>>>> personally
> > > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional
> interface,
> > > and
> > > > > add
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > > > > > >> >>>>>> `open(RecordContext)`
> > > > > > > >> >>>>>>>>>> (or
> > > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > > > > > `close()`)
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext
> > > > extends
> > > > > > > >> >>>>> ValueMapper,
> > > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload.
> Of
> > > > > course,
> > > > > > > we
> > > > > > > >> >>>> don't
> > > > > > > >> >>>>>>>>> get
> > > > > > > >> >>>>>>>>>> a
> > > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also
> sacrifices
> > > > > > Lambdas.
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what
> > the
> > > > > > better
> > > > > > > >> >>>> option
> > > > > > > >> >>>>>>>>> might
> > > > > > > >> >>>>>>>>>>>>> be.
> > > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this
> > > trade
> > > > > > off.
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang
> > Wang <
> > > > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>
> > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> > > functions+to+Streams
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>>> --
> > > > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>> --
> > > > > > > >> >>>>>>>>>> -Cheers
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>>> --
> > > > > > > >> >>>>>>>>> -- Guozhang
> > > > > > > >> >>>>>>>>>
> > > > > > > >> >>>>>>>
> > > > > > > >> >>>>>>
> > > > > > > >> >>>>>> --
> > > > > > > >> >>>>> -Cheers
> > > > > > > >> >>>>>
> > > > > > > >> >>>>> Jeyhun
> > > > > > > >> >>>>>
> > > > > > > >> >>>>
> > > > > > > >> >>> --
> > > > > > > >> >>> -Cheers
> > > > > > > >> >>>
> > > > > > > >> >>> Jeyhun
> > > > > > > >> >>>
> > > > > > > >> >>
> > > > > > > >>
> > > > > > > >> --
> > > > > > > > -Cheers
> > > > > > > >
> > > > > > > > Jeyhun
> > > > > > > >
> > > > > > > --
> > > > > > > -Cheers
> > > > > > >
> > > > > > > Jeyhun
> > > > > > >
> > > > > >
> > > > > --
> > > > > -Cheers
> > > > >
> > > > > Jeyhun
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

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

Posted by Damian Guy <da...@gmail.com>.
+1

On Thu, 21 Sep 2017 at 13:46 Guozhang Wang <wa...@gmail.com> 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 would still need to
> update your proposal for people to comment and vote on.
>
>
> Guozhang
>
>
> On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com> wrote:
>
> > +1
> >
> > One interface is cleaner.
> >
> > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com> wrote:
> >
> > > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces
> > into 1
> > > interface.
> > >
> > > Thanks,
> > > Bill
> > >
> > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <je.karimov@gmail.com
> >
> > > 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 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
> > > this
> > > > would not be a problem.
> > > >
> > > > Our initial idea was similar to :
> > > >
> > > > public abstract class RichValueMapper<K, V, VR>  implements
> > > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > > ......
> > > > }
> > > >
> > > >
> > > > So, we check the type of object, whether it is RichXXX or XXXWithKey
> > > inside
> > > > the called method and continue accordingly.
> > > >
> > > > If this is ok with the community, I would like to revert the current
> > > design
> > > > to this again.
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com>
> > wrote:
> > > >
> > > > > Hi Jeyhun,
> > > > >
> > > > > Thanks for sending out the update. I guess i was thinking more
> along
> > > the
> > > > > lines of option 2 where we collapse the RichXXXX and
> ValueXXXXWithKey
> > > 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?
> > > > >
> > > > > Thanks,
> > > > > Damian
> > > > >
> > > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > 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 free to comment on this.
> > > > > >
> > > > > > [1]
> > > > > >
> > > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > > action?pageId=73637757
> > > > > >
> > > > > > Cheers,
> > > > > > Jeyhun
> > > > > >
> > > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> > je.karimov@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > 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 <
> > > > matthias@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> 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
> > > > > > >> > for one-shot solution for overloads issue.
> > > > > > >> >
> > > > > > >> > Cheers,
> > > > > > >> > Jeyhun
> > > > > > >> >
> > > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > > damian.guy@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >
> > > > > > >> >> 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
> > > > > > overrides.
> > > > > > >> >>
> > > > > > >> >> Thanks,
> > > > > > >> >> Damian
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>> About ProcessorContext vs RecordContext, you are right. I
> > > think
> > > > I
> > > > > > >> need to
> > > > > > >> >>> implement a prototype to understand the full picture as
> some
> > > > parts
> > > > > > of
> > > > > > >> the
> > > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > > >> >>>
> > > > > > >> >>>
> > > > > > >> >>> Cheers,
> > > > > > >> >>> Jeyhun
> > > > > > >> >>>
> > > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > > damian.guy@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >>>
> > > > > > >> >>>> 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
> > lifetime
> > > of
> > > > > the
> > > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > > ProcessorContext
> > > > > to
> > > > > > a
> > > > > > >> >>>> RecordContext.
> > > > > > >> >>>> You mentioned above passing the InternalProcessorContext
> to
> > > the
> > > > > > >> init()
> > > > > > >> >>>> calls. It is internal for a reason and i think it should
> > > remain
> > > > > > that
> > > > > > >> >> way.
> > > > > > >> >>>> It might be better to move the recordContext() method
> from
> > > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > > >> >>>>
> > > > > > >> >>>> In the KIP you have an example showing:
> > > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > > >> >>>> But the interface is:
> > > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > > >> >>>>     VR apply(final V value, final RecordContext
> > > recordContext);
> > > > > > >> >>>> }
> > > > > > >> >>>> i.e., there is no init(...), besides as above this
> wouldn't
> > > > make
> > > > > > >> sense.
> > > > > > >> >>>>
> > > > > > >> >>>> Thanks,
> > > > > > >> >>>> Damian
> > > > > > >> >>>>
> > > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > > je.karimov@gmail.com
> > > > > >
> > > > > > >> >> wrote:
> > > > > > >> >>>>
> > > > > > >> >>>>> 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. Can't we
> pass
> > > > this
> > > > > > as a
> > > > > > >> >>>>> parameter to init() method of processors? Then we would
> be
> > > > able
> > > > > to
> > > > > > >> >> get
> > > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > > >> >>>>>
> > > > > > >> >>>>>
> > > > > > >> >>>>> Cheers,
> > > > > > >> >>>>> Jeyhun
> > > > > > >> >>>>>
> > > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > > >> >>> matthias@confluent.io>
> > > > > > >> >>>>> wrote:
> > > > > > >> >>>>>
> > > > > > >> >>>>>> 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
> > > > > > >> >>> will
> > > > > > >> >>>>>> be provided by the consecutive call to aggregate
> anyways.
> > > > > > >> >>>>>>
> > > > > > >> >>>>>>
> > > > > > >> >>>>>> -Matthias
> > > > > > >> >>>>>>
> > > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > > >> >>>>>>> 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 implement both. This
> > > allows
> > > > > us
> > > > > > >> >> to
> > > > > > >> >>>> cast
> > > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible
> scope.
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>> -Matthias
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > > >> >>>>>>>> continue as it is?
> > > > > > >> >>>>>>>>
> > > > > > >> >>>>>>>> Cheers,
> > > > > > >> >>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>
> > > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > > > >> >>> wangguoz@gmail.com>
> > > > > > >> >>>>>> wrote:
> > > > > > >> >>>>>>>>
> > > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>> Guozhang
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > > > >> >>>>> je.karimov@gmail.com>
> > > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also
> be
> > > > > > >> >> integrated
> > > > > > >> >>>> into
> > > > > > >> >>>>>> the
> > > > > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > > > > interface.
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>> [1]
> https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>> Cheers,
> > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > > > >> >>>>>> matthias@confluent.io>
> > > > > > >> >>>>>>>>>> wrote:
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>>> 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 information only.
> If
> > > you
> > > > > want
> > > > > > >> >> to
> > > > > > >> >>>> do
> > > > > > >> >>>>> a
> > > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > > #transform().
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to
> switch
> > > to
> > > > a
> > > > > > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple
> interface
> > > > with a
> > > > > > >> >>> single
> > > > > > >> >>>>>>>>> method
> > > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>> -Matthias
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > > >> >>>>>>>>>>>> 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 <
> > > > > > >> >>>>>> je.karimov@gmail.com
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>>> wrote:
> > > > > > >> >>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> 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, so I will not mention
> them.
> > > > > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149
> and
> > in
> > > > > this
> > > > > > >> >> KIP
> > > > > > >> >>>>>>>>>> (KIP-159)
> > > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the
> stage
> > > with
> > > > > > >> >>> KIP-149,
> > > > > > >> >>>>> in
> > > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended
> KIP-149
> > > to
> > > > > > >> >> support
> > > > > > >> >>>>> Rich
> > > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we
> provided
> > > init
> > > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we
> > should
> > > > not
> > > > > > >> >>>> provide
> > > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> > separated
> > > > the
> > > > > > >> >> two
> > > > > > >> >>>>>>>>> problems
> > > > > > >> >>>>>>>>>>> into
> > > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be
> solved
> > in
> > > > > > >> >>> parallel.
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR>
> extends
> > > > > > >> >>>> RichFunction{
> > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > > > >> >>>>>>>>>>>>>     void close();
> > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> > appConfigsWithPrefix(String
> > > > > > >> >> prefix);
> > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > > RecordContext
> > > > > {
> > > > > > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > > > > > >> >>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> As a result:
> > > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces
> can
> > be
> > > > > > >> >>> converted
> > > > > > >> >>>> to
> > > > > > >> >>>>>>>>>> their
> > > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> > > > > methods)
> > > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > > interfaces
> > > > in
> > > > > > >> >>> their
> > > > > > >> >>>>>>>>>>>>> constructors.
> > > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > > "withoutKey"
> > > > > > >> >>>>> interfaces
> > > > > > >> >>>>>>>>> to
> > > > > > >> >>>>>>>>>>> Rich
> > > > > > >> >>>>>>>>>>>>> interface while building the topology and
> > initialize
> > > > the
> > > > > > >> >>>> related
> > > > > > >> >>>>>>>>>>> processors
> > > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for
> rich
> > > > > > >> >> functions
> > > > > > >> >>> as
> > > > > > >> >>>>>> Rich
> > > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will
> just
> > > > check
> > > > > > >> >> the
> > > > > > >> >>>>> object
> > > > > > >> >>>>>>>>>> type
> > > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above
> approach
> > > > does
> > > > > > not
> > > > > > >> >>>>> support
> > > > > > >> >>>>>>>>>>> lambdas
> > > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > > >> >>> init(RecordContext),
> > > > > > >> >>>>> as
> > > > > > >> >>>>>>>>>> part
> > > > > > >> >>>>>>>>>>> of
> > > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I think
> > Rich
> > > > > > >> >>> interfaces
> > > > > > >> >>>>> are
> > > > > > >> >>>>>>>>> by
> > > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not care
> much
> > > > about
> > > > > > >> >> it.
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an
> alternative
> > > we
> > > > > > >> >>>> considered
> > > > > > >> >>>>>> was
> > > > > > >> >>>>>>>>>> to
> > > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.
> > This
> > > > > might
> > > > > > >> >>> even
> > > > > > >> >>>>>>>>> allow
> > > > > > >> >>>>>>>>>> to
> > > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> > RichFunction
> > > as
> > > > > we
> > > > > > >> >>>>> preserve
> > > > > > >> >>>>>>>>> the
> > > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> > basically
> > > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> > > function.
> > > > > > >> >> Thus,
> > > > > > >> >>> we
> > > > > > >> >>>>>>>>> cannot
> > > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc)
> > > over
> > > > > > >> >>> multiple
> > > > > > >> >>>>>> tasks
> > > > > > >> >>>>>>>>>> and
> > > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > > > #transform().
> > > > > > >> >> And
> > > > > > >> >>>>> this
> > > > > > >> >>>>>>>>>> would
> > > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > > (Rich)ValueMapperSupplier
> > > > > > >> >>> would
> > > > > > >> >>>>> not
> > > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need
> > many
> > > > new
> > > > > > >> >>>> overload
> > > > > > >> >>>>>>>>> for
> > > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> > Matthias's
> > > > > > email)
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> wrote:
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>> 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
> 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:
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> > > > context);
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> }
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> ...
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<?
> > super
> > > > V, ?
> > > > > > >> >>>> extends
> > > > > > >> >>>>>> VR>
> > > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > > mapValueswithContext(RichValueMapper
> > > > > > >> >> <?
> > > > > > >> >>>>> super
> > > > > > >> >>>>>>>>>> V, ?
> > > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > > > overloads;
> > > > > > >> >> but I
> > > > > > >> >>>>> think
> > > > > > >> >>>>>>>>>> the
> > > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> > > > > overrides
> > > > > > >> >> and
> > > > > > >> >>>> 2)
> > > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which
> can
> > > be
> > > > > > >> >> reduced
> > > > > > >> >>>> in
> > > > > > >> >>>>>> the
> > > > > > >> >>>>>>>>>>> near
> > > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > > > > worthwhile,
> > > > > > >> >> as
> > > > > > >> >>>> it
> > > > > > >> >>>>>> has
> > > > > > >> >>>>>>>>>> the
> > > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert"
> > > from
> > > > > > >> >>> non-rich
> > > > > > >> >>>>>>>>>> functions
> > > > > > >> >>>>>>>>>>>>> to
> > > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed
> > > and I
> > > > > may
> > > > > > >> >>> have
> > > > > > >> >>>>>>>>>>>>> overlooked
> > > > > > >> >>>>>>>>>>>>>> in
> > > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J.
> > Sax <
> > > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an
> > > overall
> > > > > > >> >>>> consisted
> > > > > > >> >>>>>>>>> API.
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might
> get
> > > > more
> > > > > > >> >>>> overload.
> > > > > > >> >>>>>> It
> > > > > > >> >>>>>>>>>>>>> might
> > > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > > xxxWithContext()
> > > > > > >> >>>> overload
> > > > > > >> >>>>>>>>> that
> > > > > > >> >>>>>>>>>>>>> will
> > > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if
> might
> > > get
> > > > > too
> > > > > > >> >>> messy
> > > > > > >> >>>>>>>>> having
> > > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > > ValueMapperWithContext,
> > > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder
> > > > pattern"
> > > > > > >> >> idea
> > > > > > >> >>>> as
> > > > > > >> >>>>> an
> > > > > > >> >>>>>>>>>> API
> > > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> > > > problem.
> > > > > > Not
> > > > > > >> >>> for
> > > > > > >> >>>>>>>>> simple
> > > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins
> and
> > > > > > >> >>>> aggregations.
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older
> > > > email,
> > > > > I
> > > > > > >> >> am
> > > > > > >> >>>>>>>>>> personally
> > > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface,
> > and
> > > > add
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > > > > >> >>>>>> `open(RecordContext)`
> > > > > > >> >>>>>>>>>> (or
> > > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > > > > `close()`)
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext
> > > extends
> > > > > > >> >>>>> ValueMapper,
> > > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of
> > > > course,
> > > > > > we
> > > > > > >> >>>> don't
> > > > > > >> >>>>>>>>> get
> > > > > > >> >>>>>>>>>> a
> > > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> > > > > Lambdas.
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what
> the
> > > > > better
> > > > > > >> >>>> option
> > > > > > >> >>>>>>>>> might
> > > > > > >> >>>>>>>>>>>>> be.
> > > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this
> > trade
> > > > > off.
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang
> Wang <
> > > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > > >> >>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>
> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> > functions+to+Streams
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>>> --
> > > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>> --
> > > > > > >> >>>>>>>>>> -Cheers
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>> Jeyhun
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>>> --
> > > > > > >> >>>>>>>>> -- Guozhang
> > > > > > >> >>>>>>>>>
> > > > > > >> >>>>>>>
> > > > > > >> >>>>>>
> > > > > > >> >>>>>> --
> > > > > > >> >>>>> -Cheers
> > > > > > >> >>>>>
> > > > > > >> >>>>> Jeyhun
> > > > > > >> >>>>>
> > > > > > >> >>>>
> > > > > > >> >>> --
> > > > > > >> >>> -Cheers
> > > > > > >> >>>
> > > > > > >> >>> Jeyhun
> > > > > > >> >>>
> > > > > > >> >>
> > > > > > >>
> > > > > > >> --
> > > > > > > -Cheers
> > > > > > >
> > > > > > > Jeyhun
> > > > > > >
> > > > > > --
> > > > > > -Cheers
> > > > > >
> > > > > > Jeyhun
> > > > > >
> > > > >
> > > > --
> > > > -Cheers
> > > >
> > > > Jeyhun
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
+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 vote on.


Guozhang


On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com> wrote:

> +1
>
> One interface is cleaner.
>
> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com> wrote:
>
> > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces
> into 1
> > interface.
> >
> > Thanks,
> > Bill
> >
> > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <je...@gmail.com>
> > 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 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
> > this
> > > would not be a problem.
> > >
> > > Our initial idea was similar to :
> > >
> > > public abstract class RichValueMapper<K, V, VR>  implements
> > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > ......
> > > }
> > >
> > >
> > > So, we check the type of object, whether it is RichXXX or XXXWithKey
> > inside
> > > the called method and continue accordingly.
> > >
> > > If this is ok with the community, I would like to revert the current
> > design
> > > to this again.
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com>
> wrote:
> > >
> > > > Hi Jeyhun,
> > > >
> > > > Thanks for sending out the update. I guess i was thinking more along
> > the
> > > > lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey
> > 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?
> > > >
> > > > Thanks,
> > > > Damian
> > > >
> > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com>
> > > wrote:
> > > >
> > > > > 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 free to comment on this.
> > > > >
> > > > > [1]
> > > > >
> > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > action?pageId=73637757
> > > > >
> > > > > Cheers,
> > > > > Jeyhun
> > > > >
> > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > 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 <
> > > matthias@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > >> 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
> > > > > >> > for one-shot solution for overloads issue.
> > > > > >> >
> > > > > >> > Cheers,
> > > > > >> > Jeyhun
> > > > > >> >
> > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > damian.guy@gmail.com>
> > > > > >> wrote:
> > > > > >> >
> > > > > >> >> 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
> > > > > overrides.
> > > > > >> >>
> > > > > >> >> Thanks,
> > > > > >> >> Damian
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>> About ProcessorContext vs RecordContext, you are right. I
> > think
> > > I
> > > > > >> need to
> > > > > >> >>> implement a prototype to understand the full picture as some
> > > parts
> > > > > of
> > > > > >> the
> > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > >> >>>
> > > > > >> >>>
> > > > > >> >>> Cheers,
> > > > > >> >>> Jeyhun
> > > > > >> >>>
> > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > damian.guy@gmail.com>
> > > > > >> wrote:
> > > > > >> >>>
> > > > > >> >>>> 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
> lifetime
> > of
> > > > the
> > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > ProcessorContext
> > > > to
> > > > > a
> > > > > >> >>>> RecordContext.
> > > > > >> >>>> You mentioned above passing the InternalProcessorContext to
> > the
> > > > > >> init()
> > > > > >> >>>> calls. It is internal for a reason and i think it should
> > remain
> > > > > that
> > > > > >> >> way.
> > > > > >> >>>> It might be better to move the recordContext() method from
> > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > >> >>>>
> > > > > >> >>>> In the KIP you have an example showing:
> > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > >> >>>> But the interface is:
> > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > >> >>>>     VR apply(final V value, final RecordContext
> > recordContext);
> > > > > >> >>>> }
> > > > > >> >>>> i.e., there is no init(...), besides as above this wouldn't
> > > make
> > > > > >> sense.
> > > > > >> >>>>
> > > > > >> >>>> Thanks,
> > > > > >> >>>> Damian
> > > > > >> >>>>
> > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > je.karimov@gmail.com
> > > > >
> > > > > >> >> wrote:
> > > > > >> >>>>
> > > > > >> >>>>> 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. Can't we pass
> > > this
> > > > > as a
> > > > > >> >>>>> parameter to init() method of processors? Then we would be
> > > able
> > > > to
> > > > > >> >> get
> > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > >> >>>>>
> > > > > >> >>>>>
> > > > > >> >>>>> Cheers,
> > > > > >> >>>>> Jeyhun
> > > > > >> >>>>>
> > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > >> >>> matthias@confluent.io>
> > > > > >> >>>>> wrote:
> > > > > >> >>>>>
> > > > > >> >>>>>> 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
> > > > > >> >>> will
> > > > > >> >>>>>> be provided by the consecutive call to aggregate anyways.
> > > > > >> >>>>>>
> > > > > >> >>>>>>
> > > > > >> >>>>>> -Matthias
> > > > > >> >>>>>>
> > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > >> >>>>>>> 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 implement both. This
> > allows
> > > > us
> > > > > >> >> to
> > > > > >> >>>> cast
> > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>> -Matthias
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > >> >>>>>>>> continue as it is?
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>> Cheers,
> > > > > >> >>>>>>>> Jeyhun
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > > >> >>> wangguoz@gmail.com>
> > > > > >> >>>>>> wrote:
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> Guozhang
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > > >> >>>>> je.karimov@gmail.com>
> > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> > > > > >> >> integrated
> > > > > >> >>>> into
> > > > > >> >>>>>> the
> > > > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > > > interface.
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > > >> >>>>>> matthias@confluent.io>
> > > > > >> >>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>> 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 information only. If
> > you
> > > > want
> > > > > >> >> to
> > > > > >> >>>> do
> > > > > >> >>>>> a
> > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > #transform().
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch
> > to
> > > a
> > > > > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple interface
> > > with a
> > > > > >> >>> single
> > > > > >> >>>>>>>>> method
> > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> -Matthias
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > >> >>>>>>>>>>>> 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 <
> > > > > >> >>>>>> je.karimov@gmail.com
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 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, so I will not mention them.
> > > > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and
> in
> > > > this
> > > > > >> >> KIP
> > > > > >> >>>>>>>>>> (KIP-159)
> > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage
> > with
> > > > > >> >>> KIP-149,
> > > > > >> >>>>> in
> > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149
> > to
> > > > > >> >> support
> > > > > >> >>>>> Rich
> > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided
> > init
> > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we
> should
> > > not
> > > > > >> >>>> provide
> > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> separated
> > > the
> > > > > >> >> two
> > > > > >> >>>>>>>>> problems
> > > > > >> >>>>>>>>>>> into
> > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved
> in
> > > > > >> >>> parallel.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> > > > > >> >>>> RichFunction{
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > > >> >>>>>>>>>>>>>     void close();
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> appConfigsWithPrefix(String
> > > > > >> >> prefix);
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > RecordContext
> > > > {
> > > > > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> As a result:
> > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can
> be
> > > > > >> >>> converted
> > > > > >> >>>> to
> > > > > >> >>>>>>>>>> their
> > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> > > > methods)
> > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > interfaces
> > > in
> > > > > >> >>> their
> > > > > >> >>>>>>>>>>>>> constructors.
> > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > "withoutKey"
> > > > > >> >>>>> interfaces
> > > > > >> >>>>>>>>> to
> > > > > >> >>>>>>>>>>> Rich
> > > > > >> >>>>>>>>>>>>> interface while building the topology and
> initialize
> > > the
> > > > > >> >>>> related
> > > > > >> >>>>>>>>>>> processors
> > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> > > > > >> >> functions
> > > > > >> >>> as
> > > > > >> >>>>>> Rich
> > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just
> > > check
> > > > > >> >> the
> > > > > >> >>>>> object
> > > > > >> >>>>>>>>>> type
> > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach
> > > does
> > > > > not
> > > > > >> >>>>> support
> > > > > >> >>>>>>>>>>> lambdas
> > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > >> >>> init(RecordContext),
> > > > > >> >>>>> as
> > > > > >> >>>>>>>>>> part
> > > > > >> >>>>>>>>>>> of
> > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I think
> Rich
> > > > > >> >>> interfaces
> > > > > >> >>>>> are
> > > > > >> >>>>>>>>> by
> > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not care much
> > > about
> > > > > >> >> it.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative
> > we
> > > > > >> >>>> considered
> > > > > >> >>>>>> was
> > > > > >> >>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.
> This
> > > > might
> > > > > >> >>> even
> > > > > >> >>>>>>>>> allow
> > > > > >> >>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> RichFunction
> > as
> > > > we
> > > > > >> >>>>> preserve
> > > > > >> >>>>>>>>> the
> > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> basically
> > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> > function.
> > > > > >> >> Thus,
> > > > > >> >>> we
> > > > > >> >>>>>>>>> cannot
> > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc)
> > over
> > > > > >> >>> multiple
> > > > > >> >>>>>> tasks
> > > > > >> >>>>>>>>>> and
> > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > > #transform().
> > > > > >> >> And
> > > > > >> >>>>> this
> > > > > >> >>>>>>>>>> would
> > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > (Rich)ValueMapperSupplier
> > > > > >> >>> would
> > > > > >> >>>>> not
> > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need
> many
> > > new
> > > > > >> >>>> overload
> > > > > >> >>>>>>>>> for
> > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> Matthias's
> > > > > email)
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>> 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 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:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> > > context);
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> ...
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<?
> super
> > > V, ?
> > > > > >> >>>> extends
> > > > > >> >>>>>> VR>
> > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > mapValueswithContext(RichValueMapper
> > > > > >> >> <?
> > > > > >> >>>>> super
> > > > > >> >>>>>>>>>> V, ?
> > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > > overloads;
> > > > > >> >> but I
> > > > > >> >>>>> think
> > > > > >> >>>>>>>>>> the
> > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> > > > overrides
> > > > > >> >> and
> > > > > >> >>>> 2)
> > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can
> > be
> > > > > >> >> reduced
> > > > > >> >>>> in
> > > > > >> >>>>>> the
> > > > > >> >>>>>>>>>>> near
> > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > > > worthwhile,
> > > > > >> >> as
> > > > > >> >>>> it
> > > > > >> >>>>>> has
> > > > > >> >>>>>>>>>> the
> > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert"
> > from
> > > > > >> >>> non-rich
> > > > > >> >>>>>>>>>> functions
> > > > > >> >>>>>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed
> > and I
> > > > may
> > > > > >> >>> have
> > > > > >> >>>>>>>>>>>>> overlooked
> > > > > >> >>>>>>>>>>>>>> in
> > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J.
> Sax <
> > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an
> > overall
> > > > > >> >>>> consisted
> > > > > >> >>>>>>>>> API.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get
> > > more
> > > > > >> >>>> overload.
> > > > > >> >>>>>> It
> > > > > >> >>>>>>>>>>>>> might
> > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > xxxWithContext()
> > > > > >> >>>> overload
> > > > > >> >>>>>>>>> that
> > > > > >> >>>>>>>>>>>>> will
> > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might
> > get
> > > > too
> > > > > >> >>> messy
> > > > > >> >>>>>>>>> having
> > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > ValueMapperWithContext,
> > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder
> > > pattern"
> > > > > >> >> idea
> > > > > >> >>>> as
> > > > > >> >>>>> an
> > > > > >> >>>>>>>>>> API
> > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> > > problem.
> > > > > Not
> > > > > >> >>> for
> > > > > >> >>>>>>>>> simple
> > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> > > > > >> >>>> aggregations.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older
> > > email,
> > > > I
> > > > > >> >> am
> > > > > >> >>>>>>>>>> personally
> > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface,
> and
> > > add
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > > > >> >>>>>> `open(RecordContext)`
> > > > > >> >>>>>>>>>> (or
> > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > > > `close()`)
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext
> > extends
> > > > > >> >>>>> ValueMapper,
> > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of
> > > course,
> > > > > we
> > > > > >> >>>> don't
> > > > > >> >>>>>>>>> get
> > > > > >> >>>>>>>>>> a
> > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> > > > Lambdas.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the
> > > > better
> > > > > >> >>>> option
> > > > > >> >>>>>>>>> might
> > > > > >> >>>>>>>>>>>>> be.
> > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this
> trade
> > > > off.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> functions+to+Streams
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> --
> > > > > >> >>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> --
> > > > > >> >>>>>>>>> -- Guozhang
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>
> > > > > >> >>>>>> --
> > > > > >> >>>>> -Cheers
> > > > > >> >>>>>
> > > > > >> >>>>> Jeyhun
> > > > > >> >>>>>
> > > > > >> >>>>
> > > > > >> >>> --
> > > > > >> >>> -Cheers
> > > > > >> >>>
> > > > > >> >>> Jeyhun
> > > > > >> >>>
> > > > > >> >>
> > > > > >>
> > > > > >> --
> > > > > > -Cheers
> > > > > >
> > > > > > Jeyhun
> > > > > >
> > > > > --
> > > > > -Cheers
> > > > >
> > > > > Jeyhun
> > > > >
> > > >
> > > --
> > > -Cheers
> > >
> > > Jeyhun
> > >
> >
>



-- 
-- Guozhang

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

Posted by Guozhang Wang <wa...@gmail.com>.
+1

On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu <yu...@gmail.com> wrote:

> +1
>
> One interface is cleaner.
>
> On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com> wrote:
>
> > +1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces
> into 1
> > interface.
> >
> > Thanks,
> > Bill
> >
> > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <je...@gmail.com>
> > 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 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
> > this
> > > would not be a problem.
> > >
> > > Our initial idea was similar to :
> > >
> > > public abstract class RichValueMapper<K, V, VR>  implements
> > > ValueMapperWithKey<K, V, VR>, RichFunction {
> > > ......
> > > }
> > >
> > >
> > > So, we check the type of object, whether it is RichXXX or XXXWithKey
> > inside
> > > the called method and continue accordingly.
> > >
> > > If this is ok with the community, I would like to revert the current
> > design
> > > to this again.
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com>
> wrote:
> > >
> > > > Hi Jeyhun,
> > > >
> > > > Thanks for sending out the update. I guess i was thinking more along
> > the
> > > > lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey
> > 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?
> > > >
> > > > Thanks,
> > > > Damian
> > > >
> > > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com>
> > > wrote:
> > > >
> > > > > 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 free to comment on this.
> > > > >
> > > > > [1]
> > > > >
> > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > action?pageId=73637757
> > > > >
> > > > > Cheers,
> > > > > Jeyhun
> > > > >
> > > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > 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 <
> > > matthias@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > >> 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
> > > > > >> > for one-shot solution for overloads issue.
> > > > > >> >
> > > > > >> > Cheers,
> > > > > >> > Jeyhun
> > > > > >> >
> > > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> > damian.guy@gmail.com>
> > > > > >> wrote:
> > > > > >> >
> > > > > >> >> 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
> > > > > overrides.
> > > > > >> >>
> > > > > >> >> Thanks,
> > > > > >> >> Damian
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>> About ProcessorContext vs RecordContext, you are right. I
> > think
> > > I
> > > > > >> need to
> > > > > >> >>> implement a prototype to understand the full picture as some
> > > parts
> > > > > of
> > > > > >> the
> > > > > >> >>> KIP might not be as straightforward as I thought.
> > > > > >> >>>
> > > > > >> >>>
> > > > > >> >>> Cheers,
> > > > > >> >>> Jeyhun
> > > > > >> >>>
> > > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > > damian.guy@gmail.com>
> > > > > >> wrote:
> > > > > >> >>>
> > > > > >> >>>> 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
> lifetime
> > of
> > > > the
> > > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> > ProcessorContext
> > > > to
> > > > > a
> > > > > >> >>>> RecordContext.
> > > > > >> >>>> You mentioned above passing the InternalProcessorContext to
> > the
> > > > > >> init()
> > > > > >> >>>> calls. It is internal for a reason and i think it should
> > remain
> > > > > that
> > > > > >> >> way.
> > > > > >> >>>> It might be better to move the recordContext() method from
> > > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > > >> >>>>
> > > > > >> >>>> In the KIP you have an example showing:
> > > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > > >> >>>> But the interface is:
> > > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > > >> >>>>     VR apply(final V value, final RecordContext
> > recordContext);
> > > > > >> >>>> }
> > > > > >> >>>> i.e., there is no init(...), besides as above this wouldn't
> > > make
> > > > > >> sense.
> > > > > >> >>>>
> > > > > >> >>>> Thanks,
> > > > > >> >>>> Damian
> > > > > >> >>>>
> > > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > > je.karimov@gmail.com
> > > > >
> > > > > >> >> wrote:
> > > > > >> >>>>
> > > > > >> >>>>> 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. Can't we pass
> > > this
> > > > > as a
> > > > > >> >>>>> parameter to init() method of processors? Then we would be
> > > able
> > > > to
> > > > > >> >> get
> > > > > >> >>>>> RecordContext easily with just a method call.
> > > > > >> >>>>>
> > > > > >> >>>>>
> > > > > >> >>>>> Cheers,
> > > > > >> >>>>> Jeyhun
> > > > > >> >>>>>
> > > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > > >> >>> matthias@confluent.io>
> > > > > >> >>>>> wrote:
> > > > > >> >>>>>
> > > > > >> >>>>>> 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
> > > > > >> >>> will
> > > > > >> >>>>>> be provided by the consecutive call to aggregate anyways.
> > > > > >> >>>>>>
> > > > > >> >>>>>>
> > > > > >> >>>>>> -Matthias
> > > > > >> >>>>>>
> > > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > >> >>>>>>> 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 implement both. This
> > allows
> > > > us
> > > > > >> >> to
> > > > > >> >>>> cast
> > > > > >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>> -Matthias
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > >> >>>>>>>> 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 address this issue as well or
> > > > > >> >>>>>>>> continue as it is?
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>> Cheers,
> > > > > >> >>>>>>>> Jeyhun
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > > >> >>> wangguoz@gmail.com>
> > > > > >> >>>>>> wrote:
> > > > > >> >>>>>>>>
> > > > > >> >>>>>>>>> LGTM. Thanks!
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> Guozhang
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > > >> >>>>> je.karimov@gmail.com>
> > > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> > > > > >> >> integrated
> > > > > >> >>>> into
> > > > > >> >>>>>> the
> > > > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > > > interface.
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > > >> >>>>>> matthias@confluent.io>
> > > > > >> >>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>> 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 information only. If
> > you
> > > > want
> > > > > >> >> to
> > > > > >> >>>> do
> > > > > >> >>>>> a
> > > > > >> >>>>>>>>>>> stateful computation you should rather use
> > #transform().
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch
> > to
> > > a
> > > > > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> For those reason, I advocate for a simple interface
> > > with a
> > > > > >> >>> single
> > > > > >> >>>>>>>>> method
> > > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> -Matthias
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > >> >>>>>>>>>>>> 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 <
> > > > > >> >>>>>> je.karimov@gmail.com
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 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, so I will not mention them.
> > > > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and
> in
> > > > this
> > > > > >> >> KIP
> > > > > >> >>>>>>>>>> (KIP-159)
> > > > > >> >>>>>>>>>>>>> discussion threads.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage
> > with
> > > > > >> >>> KIP-149,
> > > > > >> >>>>> in
> > > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149
> > to
> > > > > >> >> support
> > > > > >> >>>>> Rich
> > > > > >> >>>>>>>>>>>>> functions as well.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided
> > init
> > > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we
> should
> > > not
> > > > > >> >>>> provide
> > > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we
> separated
> > > the
> > > > > >> >> two
> > > > > >> >>>>>>>>> problems
> > > > > >> >>>>>>>>>>> into
> > > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved
> in
> > > > > >> >>> parallel.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> > > > > >> >>>> RichFunction{
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > > >> >>>>>>>>>>>>>     void close();
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > > >> >>>>>>>>>>>>>     String topic();
> > > > > >> >>>>>>>>>>>>>     int partition();
> > > > > >> >>>>>>>>>>>>>     long offset();
> > > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > > >> >>>>>>>>>>>>>     Map<String, Object>
> appConfigsWithPrefix(String
> > > > > >> >> prefix);
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > > RecordContext
> > > > {
> > > > > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > > > > >> >>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> As a result:
> > > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can
> be
> > > > > >> >>> converted
> > > > > >> >>>> to
> > > > > >> >>>>>>>>>> their
> > > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> > > > methods)
> > > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> > interfaces
> > > in
> > > > > >> >>> their
> > > > > >> >>>>>>>>>>>>> constructors.
> > > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > > "withoutKey"
> > > > > >> >>>>> interfaces
> > > > > >> >>>>>>>>> to
> > > > > >> >>>>>>>>>>> Rich
> > > > > >> >>>>>>>>>>>>> interface while building the topology and
> initialize
> > > the
> > > > > >> >>>> related
> > > > > >> >>>>>>>>>>> processors
> > > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> > > > > >> >> functions
> > > > > >> >>> as
> > > > > >> >>>>>> Rich
> > > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just
> > > check
> > > > > >> >> the
> > > > > >> >>>>> object
> > > > > >> >>>>>>>>>> type
> > > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach
> > > does
> > > > > not
> > > > > >> >>>>> support
> > > > > >> >>>>>>>>>>> lambdas
> > > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > > >> >>> init(RecordContext),
> > > > > >> >>>>> as
> > > > > >> >>>>>>>>>> part
> > > > > >> >>>>>>>>>>> of
> > > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I think
> Rich
> > > > > >> >>> interfaces
> > > > > >> >>>>> are
> > > > > >> >>>>>>>>> by
> > > > > >> >>>>>>>>>>>>> definition lambda-free and we should not care much
> > > about
> > > > > >> >> it.
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative
> > we
> > > > > >> >>>> considered
> > > > > >> >>>>>> was
> > > > > >> >>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.
> This
> > > > might
> > > > > >> >>> even
> > > > > >> >>>>>>>>> allow
> > > > > >> >>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name
> RichFunction
> > as
> > > > we
> > > > > >> >>>>> preserve
> > > > > >> >>>>>>>>> the
> > > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we
> basically
> > > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> > function.
> > > > > >> >> Thus,
> > > > > >> >>> we
> > > > > >> >>>>>>>>> cannot
> > > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc)
> > over
> > > > > >> >>> multiple
> > > > > >> >>>>>> tasks
> > > > > >> >>>>>>>>>> and
> > > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > > #transform().
> > > > > >> >> And
> > > > > >> >>>>> this
> > > > > >> >>>>>>>>>> would
> > > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > > (Rich)ValueMapperSupplier
> > > > > >> >>> would
> > > > > >> >>>>> not
> > > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need
> many
> > > new
> > > > > >> >>>> overload
> > > > > >> >>>>>>>>> for
> > > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from
> Matthias's
> > > > > email)
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > > >> >>>>>>>>> matthias@confluent.io
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>> 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 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:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> > > context);
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> }
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> ...
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<?
> super
> > > V, ?
> > > > > >> >>>> extends
> > > > > >> >>>>>> VR>
> > > > > >> >>>>>>>>>>>>>> mapper);
> > > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > > mapValueswithContext(RichValueMapper
> > > > > >> >> <?
> > > > > >> >>>>> super
> > > > > >> >>>>>>>>>> V, ?
> > > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > > overloads;
> > > > > >> >> but I
> > > > > >> >>>>> think
> > > > > >> >>>>>>>>>> the
> > > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> > > > overrides
> > > > > >> >> and
> > > > > >> >>>> 2)
> > > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can
> > be
> > > > > >> >> reduced
> > > > > >> >>>> in
> > > > > >> >>>>>> the
> > > > > >> >>>>>>>>>>> near
> > > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > > > worthwhile,
> > > > > >> >> as
> > > > > >> >>>> it
> > > > > >> >>>>>> has
> > > > > >> >>>>>>>>>> the
> > > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert"
> > from
> > > > > >> >>> non-rich
> > > > > >> >>>>>>>>>> functions
> > > > > >> >>>>>>>>>>>>> to
> > > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed
> > and I
> > > > may
> > > > > >> >>> have
> > > > > >> >>>>>>>>>>>>> overlooked
> > > > > >> >>>>>>>>>>>>>> in
> > > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J.
> Sax <
> > > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > > >> >>>>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> 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
> > > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an
> > overall
> > > > > >> >>>> consisted
> > > > > >> >>>>>>>>> API.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get
> > > more
> > > > > >> >>>> overload.
> > > > > >> >>>>>> It
> > > > > >> >>>>>>>>>>>>> might
> > > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > > xxxWithContext()
> > > > > >> >>>> overload
> > > > > >> >>>>>>>>> that
> > > > > >> >>>>>>>>>>>>> will
> > > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might
> > get
> > > > too
> > > > > >> >>> messy
> > > > > >> >>>>>>>>> having
> > > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > > ValueMapperWithContext,
> > > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder
> > > pattern"
> > > > > >> >> idea
> > > > > >> >>>> as
> > > > > >> >>>>> an
> > > > > >> >>>>>>>>>> API
> > > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> > > problem.
> > > > > Not
> > > > > >> >>> for
> > > > > >> >>>>>>>>> simple
> > > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> > > > > >> >>>> aggregations.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older
> > > email,
> > > > I
> > > > > >> >> am
> > > > > >> >>>>>>>>>> personally
> > > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface,
> and
> > > add
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > > > >> >>>>>> `open(RecordContext)`
> > > > > >> >>>>>>>>>> (or
> > > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > > > `close()`)
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext
> > extends
> > > > > >> >>>>> ValueMapper,
> > > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of
> > > course,
> > > > > we
> > > > > >> >>>> don't
> > > > > >> >>>>>>>>> get
> > > > > >> >>>>>>>>>> a
> > > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> > > > Lambdas.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the
> > > > better
> > > > > >> >>>> option
> > > > > >> >>>>>>>>> might
> > > > > >> >>>>>>>>>>>>> be.
> > > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this
> trade
> > > > off.
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > > >> >>>>>>>>>>>>>> wrote:
> > > > > >> >>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+
> functions+to+Streams
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > > >> >>>>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>>> --
> > > > > >> >>>>>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>> --
> > > > > >> >>>>>>>>>> -Cheers
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>> Jeyhun
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>>> --
> > > > > >> >>>>>>>>> -- Guozhang
> > > > > >> >>>>>>>>>
> > > > > >> >>>>>>>
> > > > > >> >>>>>>
> > > > > >> >>>>>> --
> > > > > >> >>>>> -Cheers
> > > > > >> >>>>>
> > > > > >> >>>>> Jeyhun
> > > > > >> >>>>>
> > > > > >> >>>>
> > > > > >> >>> --
> > > > > >> >>> -Cheers
> > > > > >> >>>
> > > > > >> >>> Jeyhun
> > > > > >> >>>
> > > > > >> >>
> > > > > >>
> > > > > >> --
> > > > > > -Cheers
> > > > > >
> > > > > > Jeyhun
> > > > > >
> > > > > --
> > > > > -Cheers
> > > > >
> > > > > Jeyhun
> > > > >
> > > >
> > > --
> > > -Cheers
> > >
> > > Jeyhun
> > >
> >
>



-- 
-- Guozhang

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

Posted by Ted Yu <yu...@gmail.com>.
+1

One interface is cleaner.

On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck <bb...@gmail.com> wrote:

> +1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces into 1
> interface.
>
> Thanks,
> Bill
>
> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <je...@gmail.com>
> 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 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
> this
> > would not be a problem.
> >
> > Our initial idea was similar to :
> >
> > public abstract class RichValueMapper<K, V, VR>  implements
> > ValueMapperWithKey<K, V, VR>, RichFunction {
> > ......
> > }
> >
> >
> > So, we check the type of object, whether it is RichXXX or XXXWithKey
> inside
> > the called method and continue accordingly.
> >
> > If this is ok with the community, I would like to revert the current
> design
> > to this again.
> >
> > Cheers,
> > Jeyhun
> >
> > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com> wrote:
> >
> > > Hi Jeyhun,
> > >
> > > Thanks for sending out the update. I guess i was thinking more along
> the
> > > lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey
> 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?
> > >
> > > Thanks,
> > > Damian
> > >
> > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com>
> > wrote:
> > >
> > > > 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 free to comment on this.
> > > >
> > > > [1]
> > > >
> > > https://cwiki.apache.org/confluence/pages/viewpage.
> > action?pageId=73637757
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <je.karimov@gmail.com
> >
> > > > wrote:
> > > >
> > > > > 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 <
> > matthias@confluent.io>
> > > > > wrote:
> > > > >
> > > > >> 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
> > > > >> > for one-shot solution for overloads issue.
> > > > >> >
> > > > >> > Cheers,
> > > > >> > Jeyhun
> > > > >> >
> > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> damian.guy@gmail.com>
> > > > >> wrote:
> > > > >> >
> > > > >> >> 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
> > > > overrides.
> > > > >> >>
> > > > >> >> Thanks,
> > > > >> >> Damian
> > > > >> >>
> > > > >> >>
> > > > >> >>> About ProcessorContext vs RecordContext, you are right. I
> think
> > I
> > > > >> need to
> > > > >> >>> implement a prototype to understand the full picture as some
> > parts
> > > > of
> > > > >> the
> > > > >> >>> KIP might not be as straightforward as I thought.
> > > > >> >>>
> > > > >> >>>
> > > > >> >>> Cheers,
> > > > >> >>> Jeyhun
> > > > >> >>>
> > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > damian.guy@gmail.com>
> > > > >> wrote:
> > > > >> >>>
> > > > >> >>>> 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 lifetime
> of
> > > the
> > > > >> >>>> Processor. Sot it doesn't make sense to cast a
> ProcessorContext
> > > to
> > > > a
> > > > >> >>>> RecordContext.
> > > > >> >>>> You mentioned above passing the InternalProcessorContext to
> the
> > > > >> init()
> > > > >> >>>> calls. It is internal for a reason and i think it should
> remain
> > > > that
> > > > >> >> way.
> > > > >> >>>> It might be better to move the recordContext() method from
> > > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > > >> >>>>
> > > > >> >>>> In the KIP you have an example showing:
> > > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > > >> >>>> But the interface is:
> > > > >> >>>> public interface RichValueMapper<V, VR> {
> > > > >> >>>>     VR apply(final V value, final RecordContext
> recordContext);
> > > > >> >>>> }
> > > > >> >>>> i.e., there is no init(...), besides as above this wouldn't
> > make
> > > > >> sense.
> > > > >> >>>>
> > > > >> >>>> Thanks,
> > > > >> >>>> Damian
> > > > >> >>>>
> > > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> > je.karimov@gmail.com
> > > >
> > > > >> >> wrote:
> > > > >> >>>>
> > > > >> >>>>> 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. Can't we pass
> > this
> > > > as a
> > > > >> >>>>> parameter to init() method of processors? Then we would be
> > able
> > > to
> > > > >> >> get
> > > > >> >>>>> RecordContext easily with just a method call.
> > > > >> >>>>>
> > > > >> >>>>>
> > > > >> >>>>> Cheers,
> > > > >> >>>>> Jeyhun
> > > > >> >>>>>
> > > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > > >> >>> matthias@confluent.io>
> > > > >> >>>>> wrote:
> > > > >> >>>>>
> > > > >> >>>>>> 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
> > > > >> >>> will
> > > > >> >>>>>> be provided by the consecutive call to aggregate anyways.
> > > > >> >>>>>>
> > > > >> >>>>>>
> > > > >> >>>>>> -Matthias
> > > > >> >>>>>>
> > > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > >> >>>>>>> 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 implement both. This
> allows
> > > us
> > > > >> >> to
> > > > >> >>>> cast
> > > > >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> > > > >> >>>>>>>
> > > > >> >>>>>>>
> > > > >> >>>>>>> -Matthias
> > > > >> >>>>>>>
> > > > >> >>>>>>>
> > > > >> >>>>>>>
> > > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > >> >>>>>>>> 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 address this issue as well or
> > > > >> >>>>>>>> continue as it is?
> > > > >> >>>>>>>>
> > > > >> >>>>>>>> Cheers,
> > > > >> >>>>>>>> Jeyhun
> > > > >> >>>>>>>>
> > > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > > >> >>> wangguoz@gmail.com>
> > > > >> >>>>>> wrote:
> > > > >> >>>>>>>>
> > > > >> >>>>>>>>> LGTM. Thanks!
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>> Guozhang
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > >> >>>>> je.karimov@gmail.com>
> > > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> > > > >> >> integrated
> > > > >> >>>> into
> > > > >> >>>>>> the
> > > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > > interface.
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>> Cheers,
> > > > >> >>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > >> >>>>>> matthias@confluent.io>
> > > > >> >>>>>>>>>> wrote:
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>>> 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 information only. If
> you
> > > want
> > > > >> >> to
> > > > >> >>>> do
> > > > >> >>>>> a
> > > > >> >>>>>>>>>>> stateful computation you should rather use
> #transform().
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch
> to
> > a
> > > > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>> For those reason, I advocate for a simple interface
> > with a
> > > > >> >>> single
> > > > >> >>>>>>>>> method
> > > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>> -Matthias
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > >> >>>>>>>>>>>> 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 <
> > > > >> >>>>>> je.karimov@gmail.com
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>>> wrote:
> > > > >> >>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> 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, so I will not mention them.
> > > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in
> > > this
> > > > >> >> KIP
> > > > >> >>>>>>>>>> (KIP-159)
> > > > >> >>>>>>>>>>>>> discussion threads.
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage
> with
> > > > >> >>> KIP-149,
> > > > >> >>>>> in
> > > > >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149
> to
> > > > >> >> support
> > > > >> >>>>> Rich
> > > > >> >>>>>>>>>>>>> functions as well.
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided
> init
> > > > >> >>>>>>>>>>> (ProcessorContext)
> > > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should
> > not
> > > > >> >>>> provide
> > > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated
> > the
> > > > >> >> two
> > > > >> >>>>>>>>> problems
> > > > >> >>>>>>>>>>> into
> > > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
> > > > >> >>> parallel.
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > > >> >>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> > > > >> >>>> RichFunction{
> > > > >> >>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > > >> >>>>>>>>>>>>>     void close();
> > > > >> >>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > > >> >>>>>>>>>>>>>     String applicationId();
> > > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > > >> >>>>>>>>>>>>>     String topic();
> > > > >> >>>>>>>>>>>>>     int partition();
> > > > >> >>>>>>>>>>>>>     long offset();
> > > > >> >>>>>>>>>>>>>     long timestamp();
> > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> > > > >> >> prefix);
> > > > >> >>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> > RecordContext
> > > {
> > > > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > > > >> >>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> As a result:
> > > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> > > > >> >>> converted
> > > > >> >>>> to
> > > > >> >>>>>>>>>> their
> > > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> > > methods)
> > > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich
> interfaces
> > in
> > > > >> >>> their
> > > > >> >>>>>>>>>>>>> constructors.
> > > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> > "withoutKey"
> > > > >> >>>>> interfaces
> > > > >> >>>>>>>>> to
> > > > >> >>>>>>>>>>> Rich
> > > > >> >>>>>>>>>>>>> interface while building the topology and initialize
> > the
> > > > >> >>>> related
> > > > >> >>>>>>>>>>> processors
> > > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> > > > >> >> functions
> > > > >> >>> as
> > > > >> >>>>>> Rich
> > > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just
> > check
> > > > >> >> the
> > > > >> >>>>> object
> > > > >> >>>>>>>>>> type
> > > > >> >>>>>>>>>>>>> and act accordingly.
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach
> > does
> > > > not
> > > > >> >>>>> support
> > > > >> >>>>>>>>>>> lambdas
> > > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > > >> >>> init(RecordContext),
> > > > >> >>>>> as
> > > > >> >>>>>>>>>> part
> > > > >> >>>>>>>>>>> of
> > > > >> >>>>>>>>>>>>> Rich interfaces.
> > > > >> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
> > > > >> >>> interfaces
> > > > >> >>>>> are
> > > > >> >>>>>>>>> by
> > > > >> >>>>>>>>>>>>> definition lambda-free and we should not care much
> > about
> > > > >> >> it.
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative
> we
> > > > >> >>>> considered
> > > > >> >>>>>> was
> > > > >> >>>>>>>>>> to
> > > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This
> > > might
> > > > >> >>> even
> > > > >> >>>>>>>>> allow
> > > > >> >>>>>>>>>> to
> > > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction
> as
> > > we
> > > > >> >>>>> preserve
> > > > >> >>>>>>>>> the
> > > > >> >>>>>>>>>>>>> nature of being a function.
> > > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
> > > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a
> function.
> > > > >> >> Thus,
> > > > >> >>> we
> > > > >> >>>>>>>>> cannot
> > > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc)
> over
> > > > >> >>> multiple
> > > > >> >>>>>> tasks
> > > > >> >>>>>>>>>> and
> > > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > > #transform().
> > > > >> >> And
> > > > >> >>>>> this
> > > > >> >>>>>>>>>> would
> > > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > > (Rich)ValueMapperSupplier
> > > > >> >>> would
> > > > >> >>>>> not
> > > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many
> > new
> > > > >> >>>> overload
> > > > >> >>>>>>>>> for
> > > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's
> > > > email)
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> Cheers,
> > > > >> >>>>>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > >> >>>>>>>>> matthias@confluent.io
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> wrote:
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>> 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 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:
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> ---------------------------
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> > context);
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> }
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> ...
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super
> > V, ?
> > > > >> >>>> extends
> > > > >> >>>>>> VR>
> > > > >> >>>>>>>>>>>>>> mapper);
> > > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > > mapValueswithContext(RichValueMapper
> > > > >> >> <?
> > > > >> >>>>> super
> > > > >> >>>>>>>>>> V, ?
> > > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> -------------------------------
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> > overloads;
> > > > >> >> but I
> > > > >> >>>>> think
> > > > >> >>>>>>>>>> the
> > > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> > > overrides
> > > > >> >> and
> > > > >> >>>> 2)
> > > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can
> be
> > > > >> >> reduced
> > > > >> >>>> in
> > > > >> >>>>>> the
> > > > >> >>>>>>>>>>> near
> > > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > > worthwhile,
> > > > >> >> as
> > > > >> >>>> it
> > > > >> >>>>>> has
> > > > >> >>>>>>>>>> the
> > > > >> >>>>>>>>>>>>>>> following benefits:
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert"
> from
> > > > >> >>> non-rich
> > > > >> >>>>>>>>>> functions
> > > > >> >>>>>>>>>>>>> to
> > > > >> >>>>>>>>>>>>>>> rich functions)
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed
> and I
> > > may
> > > > >> >>> have
> > > > >> >>>>>>>>>>>>> overlooked
> > > > >> >>>>>>>>>>>>>> in
> > > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> Guozhang
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > > >> >>>>>>>>>>>>>>> wrote:
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> 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
> > > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an
> overall
> > > > >> >>>> consisted
> > > > >> >>>>>>>>> API.
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get
> > more
> > > > >> >>>> overload.
> > > > >> >>>>>> It
> > > > >> >>>>>>>>>>>>> might
> > > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> > xxxWithContext()
> > > > >> >>>> overload
> > > > >> >>>>>>>>> that
> > > > >> >>>>>>>>>>>>> will
> > > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might
> get
> > > too
> > > > >> >>> messy
> > > > >> >>>>>>>>> having
> > > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > > ValueMapperWithContext,
> > > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder
> > pattern"
> > > > >> >> idea
> > > > >> >>>> as
> > > > >> >>>>> an
> > > > >> >>>>>>>>>> API
> > > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> > problem.
> > > > Not
> > > > >> >>> for
> > > > >> >>>>>>>>> simple
> > > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> > > > >> >>>> aggregations.
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older
> > email,
> > > I
> > > > >> >> am
> > > > >> >>>>>>>>>> personally
> > > > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and
> > add
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > > >> >>>>>> `open(RecordContext)`
> > > > >> >>>>>>>>>> (or
> > > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > > `close()`)
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext
> extends
> > > > >> >>>>> ValueMapper,
> > > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of
> > course,
> > > > we
> > > > >> >>>> don't
> > > > >> >>>>>>>>> get
> > > > >> >>>>>>>>>> a
> > > > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> > > Lambdas.
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the
> > > better
> > > > >> >>>> option
> > > > >> >>>>>>>>> might
> > > > >> >>>>>>>>>>>>> be.
> > > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade
> > > off.
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> -Matthias
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > >> >>>>>>>>>>>>>>>>> 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.
> > > > >> >>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > > >> >>>>>>>>> wangguoz@gmail.com>
> > > > >> >>>>>>>>>>>>>> wrote:
> > > > >> >>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>>>>>>>>>> --
> > > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>>> --
> > > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > > >> >>>>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>>> --
> > > > >> >>>>>>>>>>>>> -Cheers
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>
> > > > >> >>>>>>>>>>>>
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>> --
> > > > >> >>>>>>>>>> -Cheers
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>> Jeyhun
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>>> --
> > > > >> >>>>>>>>> -- Guozhang
> > > > >> >>>>>>>>>
> > > > >> >>>>>>>
> > > > >> >>>>>>
> > > > >> >>>>>> --
> > > > >> >>>>> -Cheers
> > > > >> >>>>>
> > > > >> >>>>> Jeyhun
> > > > >> >>>>>
> > > > >> >>>>
> > > > >> >>> --
> > > > >> >>> -Cheers
> > > > >> >>>
> > > > >> >>> Jeyhun
> > > > >> >>>
> > > > >> >>
> > > > >>
> > > > >> --
> > > > > -Cheers
> > > > >
> > > > > Jeyhun
> > > > >
> > > > --
> > > > -Cheers
> > > >
> > > > Jeyhun
> > > >
> > >
> > --
> > -Cheers
> >
> > Jeyhun
> >
>

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

Posted by Bill Bejeck <bb...@gmail.com>.
+1 for me on collapsing the RichXXXX and ValueXXXXWithKey interfaces into 1
interface.

Thanks,
Bill

On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov <je...@gmail.com>
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 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 this
> would not be a problem.
>
> Our initial idea was similar to :
>
> public abstract class RichValueMapper<K, V, VR>  implements
> ValueMapperWithKey<K, V, VR>, RichFunction {
> ......
> }
>
>
> So, we check the type of object, whether it is RichXXX or XXXWithKey inside
> the called method and continue accordingly.
>
> If this is ok with the community, I would like to revert the current design
> to this again.
>
> Cheers,
> Jeyhun
>
> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com> wrote:
>
> > Hi Jeyhun,
> >
> > Thanks for sending out the update. I guess i was thinking more along the
> > lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey 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?
> >
> > Thanks,
> > Damian
> >
> > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com>
> wrote:
> >
> > > 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 free to comment on this.
> > >
> > > [1]
> > >
> > https://cwiki.apache.org/confluence/pages/viewpage.
> action?pageId=73637757
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <je...@gmail.com>
> > > wrote:
> > >
> > > > 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 <
> matthias@confluent.io>
> > > > wrote:
> > > >
> > > >> 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
> > > >> > for one-shot solution for overloads issue.
> > > >> >
> > > >> > Cheers,
> > > >> > Jeyhun
> > > >> >
> > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> >> 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
> > > overrides.
> > > >> >>
> > > >> >> Thanks,
> > > >> >> Damian
> > > >> >>
> > > >> >>
> > > >> >>> About ProcessorContext vs RecordContext, you are right. I think
> I
> > > >> need to
> > > >> >>> implement a prototype to understand the full picture as some
> parts
> > > of
> > > >> the
> > > >> >>> KIP might not be as straightforward as I thought.
> > > >> >>>
> > > >> >>>
> > > >> >>> Cheers,
> > > >> >>> Jeyhun
> > > >> >>>
> > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> damian.guy@gmail.com>
> > > >> wrote:
> > > >> >>>
> > > >> >>>> 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 lifetime of
> > the
> > > >> >>>> Processor. Sot it doesn't make sense to cast a ProcessorContext
> > to
> > > a
> > > >> >>>> RecordContext.
> > > >> >>>> You mentioned above passing the InternalProcessorContext to the
> > > >> init()
> > > >> >>>> calls. It is internal for a reason and i think it should remain
> > > that
> > > >> >> way.
> > > >> >>>> It might be better to move the recordContext() method from
> > > >> >>>> InternalProcessorContext to ProcessorContext.
> > > >> >>>>
> > > >> >>>> In the KIP you have an example showing:
> > > >> >>>> richMapper.init((RecordContext) processorContext);
> > > >> >>>> But the interface is:
> > > >> >>>> public interface RichValueMapper<V, VR> {
> > > >> >>>>     VR apply(final V value, final RecordContext recordContext);
> > > >> >>>> }
> > > >> >>>> i.e., there is no init(...), besides as above this wouldn't
> make
> > > >> sense.
> > > >> >>>>
> > > >> >>>> Thanks,
> > > >> >>>> Damian
> > > >> >>>>
> > > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <
> je.karimov@gmail.com
> > >
> > > >> >> wrote:
> > > >> >>>>
> > > >> >>>>> 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. Can't we pass
> this
> > > as a
> > > >> >>>>> parameter to init() method of processors? Then we would be
> able
> > to
> > > >> >> get
> > > >> >>>>> RecordContext easily with just a method call.
> > > >> >>>>>
> > > >> >>>>>
> > > >> >>>>> Cheers,
> > > >> >>>>> Jeyhun
> > > >> >>>>>
> > > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > > >> >>> matthias@confluent.io>
> > > >> >>>>> wrote:
> > > >> >>>>>
> > > >> >>>>>> 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
> > > >> >>> will
> > > >> >>>>>> be provided by the consecutive call to aggregate anyways.
> > > >> >>>>>>
> > > >> >>>>>>
> > > >> >>>>>> -Matthias
> > > >> >>>>>>
> > > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > >> >>>>>>> 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 implement both. This allows
> > us
> > > >> >> to
> > > >> >>>> cast
> > > >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> > > >> >>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>> -Matthias
> > > >> >>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > >> >>>>>>>> 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 address this issue as well or
> > > >> >>>>>>>> continue as it is?
> > > >> >>>>>>>>
> > > >> >>>>>>>> Cheers,
> > > >> >>>>>>>> Jeyhun
> > > >> >>>>>>>>
> > > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > > >> >>> wangguoz@gmail.com>
> > > >> >>>>>> wrote:
> > > >> >>>>>>>>
> > > >> >>>>>>>>> LGTM. Thanks!
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Guozhang
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > >> >>>>> je.karimov@gmail.com>
> > > >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> > > >> >> integrated
> > > >> >>>> into
> > > >> >>>>>> the
> > > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> > interface.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Cheers,
> > > >> >>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > >> >>>>>> matthias@confluent.io>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>> 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 information only. If you
> > want
> > > >> >> to
> > > >> >>>> do
> > > >> >>>>> a
> > > >> >>>>>>>>>>> stateful computation you should rather use #transform().
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch to
> a
> > > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> For those reason, I advocate for a simple interface
> with a
> > > >> >>> single
> > > >> >>>>>>>>> method
> > > >> >>>>>>>>>>> that passes in a RecordContext object.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> -Matthias
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > >> >>>>>>>>>>>> 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 <
> > > >> >>>>>> je.karimov@gmail.com
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>> wrote:
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> 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, so I will not mention them.
> > > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in
> > this
> > > >> >> KIP
> > > >> >>>>>>>>>> (KIP-159)
> > > >> >>>>>>>>>>>>> discussion threads.
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
> > > >> >>> KIP-149,
> > > >> >>>>> in
> > > >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
> > > >> >> support
> > > >> >>>>> Rich
> > > >> >>>>>>>>>>>>> functions as well.
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
> > > >> >>>>>>>>>>> (ProcessorContext)
> > > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should
> not
> > > >> >>>> provide
> > > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated
> the
> > > >> >> two
> > > >> >>>>>>>>> problems
> > > >> >>>>>>>>>>> into
> > > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
> > > >> >>> parallel.
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> - One approach we considered was :
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > > >> >>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> > > >> >>>> RichFunction{
> > > >> >>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> public interface RichFunction {
> > > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > > >> >>>>>>>>>>>>>     void close();
> > > >> >>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> public interface RecordContext {
> > > >> >>>>>>>>>>>>>     String applicationId();
> > > >> >>>>>>>>>>>>>     TaskId taskId();
> > > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > > >> >>>>>>>>>>>>>     String topic();
> > > >> >>>>>>>>>>>>>     int partition();
> > > >> >>>>>>>>>>>>>     long offset();
> > > >> >>>>>>>>>>>>>     long timestamp();
> > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > > >> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> > > >> >> prefix);
> > > >> >>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> public interface ProcessorContext extends
> RecordContext
> > {
> > > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > > >> >>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> As a result:
> > > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> > > >> >>> converted
> > > >> >>>> to
> > > >> >>>>>>>>>> their
> > > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> > methods)
> > > >> >>>>>>>>>>>>> *. All related Processors will accept Rich interfaces
> in
> > > >> >>> their
> > > >> >>>>>>>>>>>>> constructors.
> > > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or
> "withoutKey"
> > > >> >>>>> interfaces
> > > >> >>>>>>>>> to
> > > >> >>>>>>>>>>> Rich
> > > >> >>>>>>>>>>>>> interface while building the topology and initialize
> the
> > > >> >>>> related
> > > >> >>>>>>>>>>> processors
> > > >> >>>>>>>>>>>>> with Rich interfaces only.
> > > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> > > >> >> functions
> > > >> >>> as
> > > >> >>>>>> Rich
> > > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just
> check
> > > >> >> the
> > > >> >>>>> object
> > > >> >>>>>>>>>> type
> > > >> >>>>>>>>>>>>> and act accordingly.
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach
> does
> > > not
> > > >> >>>>> support
> > > >> >>>>>>>>>>> lambdas
> > > >> >>>>>>>>>>>>> so we should support only one method, only
> > > >> >>> init(RecordContext),
> > > >> >>>>> as
> > > >> >>>>>>>>>> part
> > > >> >>>>>>>>>>> of
> > > >> >>>>>>>>>>>>> Rich interfaces.
> > > >> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
> > > >> >>> interfaces
> > > >> >>>>> are
> > > >> >>>>>>>>> by
> > > >> >>>>>>>>>>>>> definition lambda-free and we should not care much
> about
> > > >> >> it.
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> > > >> >>>> considered
> > > >> >>>>>> was
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This
> > might
> > > >> >>> even
> > > >> >>>>>>>>> allow
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as
> > we
> > > >> >>>>> preserve
> > > >> >>>>>>>>> the
> > > >> >>>>>>>>>>>>> nature of being a function.
> > > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
> > > >> >>>>>>>>>>>>> allow users to have an in-memory state for a function.
> > > >> >> Thus,
> > > >> >>> we
> > > >> >>>>>>>>> cannot
> > > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
> > > >> >>> multiple
> > > >> >>>>>> tasks
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> > #transform().
> > > >> >> And
> > > >> >>>>> this
> > > >> >>>>>>>>>> would
> > > >> >>>>>>>>>>>>> "break the flow" of the API, as
> > (Rich)ValueMapperSupplier
> > > >> >>> would
> > > >> >>>>> not
> > > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many
> new
> > > >> >>>> overload
> > > >> >>>>>>>>> for
> > > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's
> > > email)
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> Cheers,
> > > >> >>>>>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > >> >>>>>>>>> matthias@confluent.io
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>>>> wrote:
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>> 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 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:
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> ---------------------------
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext
> context);
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> }
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> ...
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> // then in KStreams
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super
> V, ?
> > > >> >>>> extends
> > > >> >>>>>> VR>
> > > >> >>>>>>>>>>>>>> mapper);
> > > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> > mapValueswithContext(RichValueMapper
> > > >> >> <?
> > > >> >>>>> super
> > > >> >>>>>>>>>> V, ?
> > > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> -------------------------------
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more
> overloads;
> > > >> >> but I
> > > >> >>>>> think
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> > overrides
> > > >> >> and
> > > >> >>>> 2)
> > > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
> > > >> >> reduced
> > > >> >>>> in
> > > >> >>>>>> the
> > > >> >>>>>>>>>>> near
> > > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> > worthwhile,
> > > >> >> as
> > > >> >>>> it
> > > >> >>>>>> has
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>>>>>> following benefits:
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
> > > >> >>> non-rich
> > > >> >>>>>>>>>> functions
> > > >> >>>>>>>>>>>>> to
> > > >> >>>>>>>>>>>>>>> rich functions)
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed and I
> > may
> > > >> >>> have
> > > >> >>>>>>>>>>>>> overlooked
> > > >> >>>>>>>>>>>>>> in
> > > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> Guozhang
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > >> >>>>>>>>>>>>> matthias@confluent.io>
> > > >> >>>>>>>>>>>>>>> wrote:
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> 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
> > > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
> > > >> >>>> consisted
> > > >> >>>>>>>>> API.
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get
> more
> > > >> >>>> overload.
> > > >> >>>>>> It
> > > >> >>>>>>>>>>>>> might
> > > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single
> xxxWithContext()
> > > >> >>>> overload
> > > >> >>>>>>>>> that
> > > >> >>>>>>>>>>>>> will
> > > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get
> > too
> > > >> >>> messy
> > > >> >>>>>>>>> having
> > > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > > ValueMapperWithContext,
> > > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder
> pattern"
> > > >> >> idea
> > > >> >>>> as
> > > >> >>>>> an
> > > >> >>>>>>>>>> API
> > > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload
> problem.
> > > Not
> > > >> >>> for
> > > >> >>>>>>>>> simple
> > > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> > > >> >>>> aggregations.
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older
> email,
> > I
> > > >> >> am
> > > >> >>>>>>>>>> personally
> > > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and
> add
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > > >> >>>>>> `open(RecordContext)`
> > > >> >>>>>>>>>> (or
> > > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> > `close()`)
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> > > >> >>>>> ValueMapper,
> > > >> >>>>>>>>>>>>>>>> WithRecordContext
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of
> course,
> > > we
> > > >> >>>> don't
> > > >> >>>>>>>>> get
> > > >> >>>>>>>>>> a
> > > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> > Lambdas.
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the
> > better
> > > >> >>>> option
> > > >> >>>>>>>>> might
> > > >> >>>>>>>>>>>>> be.
> > > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade
> > off.
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> -Matthias
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > >> >>>>>>>>>>>>>>>>> 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.
> > > >> >>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>> Cheers,
> > > >> >>>>>>>>>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > >> >>>>>>>>> wangguoz@gmail.com>
> > > >> >>>>>>>>>>>>>> wrote:
> > > >> >>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>> 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 <
> > > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>> [1]
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > > >> >>>>>>>>>>>>>>>>>>> [2]
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>>>>>>>>>> --
> > > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>>> --
> > > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > > >> >>>>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>>> --
> > > >> >>>>>>>>>>>>> -Cheers
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>>>>
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> --
> > > >> >>>>>>>>>> -Cheers
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Jeyhun
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> --
> > > >> >>>>>>>>> -- Guozhang
> > > >> >>>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>>> --
> > > >> >>>>> -Cheers
> > > >> >>>>>
> > > >> >>>>> Jeyhun
> > > >> >>>>>
> > > >> >>>>
> > > >> >>> --
> > > >> >>> -Cheers
> > > >> >>>
> > > >> >>> Jeyhun
> > > >> >>>
> > > >> >>
> > > >>
> > > >> --
> > > > -Cheers
> > > >
> > > > Jeyhun
> > > >
> > > --
> > > -Cheers
> > >
> > > Jeyhun
> > >
> >
> --
> -Cheers
>
> Jeyhun
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 this
would not be a problem.

Our initial idea was similar to :

public abstract class RichValueMapper<K, V, VR>  implements
ValueMapperWithKey<K, V, VR>, RichFunction {
......
}


So, we check the type of object, whether it is RichXXX or XXXWithKey inside
the called method and continue accordingly.

If this is ok with the community, I would like to revert the current design
to this again.

Cheers,
Jeyhun

On Wed, Sep 13, 2017 at 3:02 PM Damian Guy <da...@gmail.com> wrote:

> Hi Jeyhun,
>
> Thanks for sending out the update. I guess i was thinking more along the
> lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey 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?
>
> Thanks,
> Damian
>
> On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com> wrote:
>
> > 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 free to comment on this.
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=73637757
> >
> > Cheers,
> > Jeyhun
> >
> > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <je...@gmail.com>
> > wrote:
> >
> > > 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 <ma...@confluent.io>
> > > wrote:
> > >
> > >> 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
> > >> > for one-shot solution for overloads issue.
> > >> >
> > >> > Cheers,
> > >> > Jeyhun
> > >> >
> > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com>
> > >> wrote:
> > >> >
> > >> >> 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
> > overrides.
> > >> >>
> > >> >> Thanks,
> > >> >> Damian
> > >> >>
> > >> >>
> > >> >>> About ProcessorContext vs RecordContext, you are right. I think I
> > >> need to
> > >> >>> implement a prototype to understand the full picture as some parts
> > of
> > >> the
> > >> >>> KIP might not be as straightforward as I thought.
> > >> >>>
> > >> >>>
> > >> >>> Cheers,
> > >> >>> Jeyhun
> > >> >>>
> > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com>
> > >> wrote:
> > >> >>>
> > >> >>>> 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 lifetime of
> the
> > >> >>>> Processor. Sot it doesn't make sense to cast a ProcessorContext
> to
> > a
> > >> >>>> RecordContext.
> > >> >>>> You mentioned above passing the InternalProcessorContext to the
> > >> init()
> > >> >>>> calls. It is internal for a reason and i think it should remain
> > that
> > >> >> way.
> > >> >>>> It might be better to move the recordContext() method from
> > >> >>>> InternalProcessorContext to ProcessorContext.
> > >> >>>>
> > >> >>>> In the KIP you have an example showing:
> > >> >>>> richMapper.init((RecordContext) processorContext);
> > >> >>>> But the interface is:
> > >> >>>> public interface RichValueMapper<V, VR> {
> > >> >>>>     VR apply(final V value, final RecordContext recordContext);
> > >> >>>> }
> > >> >>>> i.e., there is no init(...), besides as above this wouldn't make
> > >> sense.
> > >> >>>>
> > >> >>>> Thanks,
> > >> >>>> Damian
> > >> >>>>
> > >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je.karimov@gmail.com
> >
> > >> >> wrote:
> > >> >>>>
> > >> >>>>> 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. Can't we pass this
> > as a
> > >> >>>>> parameter to init() method of processors? Then we would be able
> to
> > >> >> get
> > >> >>>>> RecordContext easily with just a method call.
> > >> >>>>>
> > >> >>>>>
> > >> >>>>> Cheers,
> > >> >>>>> Jeyhun
> > >> >>>>>
> > >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > >> >>> matthias@confluent.io>
> > >> >>>>> wrote:
> > >> >>>>>
> > >> >>>>>> 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
> > >> >>> will
> > >> >>>>>> be provided by the consecutive call to aggregate anyways.
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>>> -Matthias
> > >> >>>>>>
> > >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > >> >>>>>>> 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 implement both. This allows
> us
> > >> >> to
> > >> >>>> cast
> > >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> -Matthias
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > >> >>>>>>>> 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 address this issue as well or
> > >> >>>>>>>> continue as it is?
> > >> >>>>>>>>
> > >> >>>>>>>> Cheers,
> > >> >>>>>>>> Jeyhun
> > >> >>>>>>>>
> > >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > >> >>> wangguoz@gmail.com>
> > >> >>>>>> wrote:
> > >> >>>>>>>>
> > >> >>>>>>>>> LGTM. Thanks!
> > >> >>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>> Guozhang
> > >> >>>>>>>>>
> > >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > >> >>>>> je.karimov@gmail.com>
> > >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> > >> >> integrated
> > >> >>>> into
> > >> >>>>>> the
> > >> >>>>>>>>>> KIP by adding related method inside RecordContext
> interface.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Cheers,
> > >> >>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > >> >>>>>> matthias@confluent.io>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>> 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 information only. If you
> want
> > >> >> to
> > >> >>>> do
> > >> >>>>> a
> > >> >>>>>>>>>>> stateful computation you should rather use #transform().
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch to a
> > >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> For those reason, I advocate for a simple interface with a
> > >> >>> single
> > >> >>>>>>>>> method
> > >> >>>>>>>>>>> that passes in a RecordContext object.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> -Matthias
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > >> >>>>>>>>>>>> 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 <
> > >> >>>>>> je.karimov@gmail.com
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>> wrote:
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>>> 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, so I will not mention them.
> > >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in
> this
> > >> >> KIP
> > >> >>>>>>>>>> (KIP-159)
> > >> >>>>>>>>>>>>> discussion threads.
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
> > >> >>> KIP-149,
> > >> >>>>> in
> > >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
> > >> >> support
> > >> >>>>> Rich
> > >> >>>>>>>>>>>>> functions as well.
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
> > >> >>>>>>>>>>> (ProcessorContext)
> > >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should not
> > >> >>>> provide
> > >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated the
> > >> >> two
> > >> >>>>>>>>> problems
> > >> >>>>>>>>>>> into
> > >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
> > >> >>> parallel.
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> - One approach we considered was :
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> > >> >>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> > >> >>>> RichFunction{
> > >> >>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> public interface RichFunction {
> > >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> > >> >>>>>>>>>>>>>     void close();
> > >> >>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> public interface RecordContext {
> > >> >>>>>>>>>>>>>     String applicationId();
> > >> >>>>>>>>>>>>>     TaskId taskId();
> > >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> > >> >>>>>>>>>>>>>     String topic();
> > >> >>>>>>>>>>>>>     int partition();
> > >> >>>>>>>>>>>>>     long offset();
> > >> >>>>>>>>>>>>>     long timestamp();
> > >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> > >> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> > >> >> prefix);
> > >> >>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> public interface ProcessorContext extends RecordContext
> {
> > >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> > >> >>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> As a result:
> > >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> > >> >>> converted
> > >> >>>> to
> > >> >>>>>>>>>> their
> > >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close()
> methods)
> > >> >>>>>>>>>>>>> *. All related Processors will accept Rich interfaces in
> > >> >>> their
> > >> >>>>>>>>>>>>> constructors.
> > >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> > >> >>>>> interfaces
> > >> >>>>>>>>> to
> > >> >>>>>>>>>>> Rich
> > >> >>>>>>>>>>>>> interface while building the topology and initialize the
> > >> >>>> related
> > >> >>>>>>>>>>> processors
> > >> >>>>>>>>>>>>> with Rich interfaces only.
> > >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> > >> >> functions
> > >> >>> as
> > >> >>>>>> Rich
> > >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just check
> > >> >> the
> > >> >>>>> object
> > >> >>>>>>>>>> type
> > >> >>>>>>>>>>>>> and act accordingly.
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach does
> > not
> > >> >>>>> support
> > >> >>>>>>>>>>> lambdas
> > >> >>>>>>>>>>>>> so we should support only one method, only
> > >> >>> init(RecordContext),
> > >> >>>>> as
> > >> >>>>>>>>>> part
> > >> >>>>>>>>>>> of
> > >> >>>>>>>>>>>>> Rich interfaces.
> > >> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
> > >> >>> interfaces
> > >> >>>>> are
> > >> >>>>>>>>> by
> > >> >>>>>>>>>>>>> definition lambda-free and we should not care much about
> > >> >> it.
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> > >> >>>> considered
> > >> >>>>>> was
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This
> might
> > >> >>> even
> > >> >>>>>>>>> allow
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as
> we
> > >> >>>>> preserve
> > >> >>>>>>>>> the
> > >> >>>>>>>>>>>>> nature of being a function.
> > >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
> > >> >>>>>>>>>>>>> allow users to have an in-memory state for a function.
> > >> >> Thus,
> > >> >>> we
> > >> >>>>>>>>> cannot
> > >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
> > >> >>> multiple
> > >> >>>>>> tasks
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>>>> we would need a supplier pattern similar to
> #transform().
> > >> >> And
> > >> >>>>> this
> > >> >>>>>>>>>> would
> > >> >>>>>>>>>>>>> "break the flow" of the API, as
> (Rich)ValueMapperSupplier
> > >> >>> would
> > >> >>>>> not
> > >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many new
> > >> >>>> overload
> > >> >>>>>>>>> for
> > >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's
> > email)
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> Cheers,
> > >> >>>>>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > >> >>>>>>>>> matthias@confluent.io
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>>>> wrote:
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>> 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 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:
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> ---------------------------
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext context);
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> }
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> ...
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> // then in KStreams
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> > >> >>>> extends
> > >> >>>>>> VR>
> > >> >>>>>>>>>>>>>> mapper);
> > >> >>>>>>>>>>>>>>> <VR> KStream<K, VR>
> mapValueswithContext(RichValueMapper
> > >> >> <?
> > >> >>>>> super
> > >> >>>>>>>>>> V, ?
> > >> >>>>>>>>>>>>>>> extends VR> mapper);
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> -------------------------------
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> The caveat is that it will introduces more overloads;
> > >> >> but I
> > >> >>>>> think
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde
> overrides
> > >> >> and
> > >> >>>> 2)
> > >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
> > >> >> reduced
> > >> >>>> in
> > >> >>>>>> the
> > >> >>>>>>>>>>> near
> > >> >>>>>>>>>>>>>>> future, and I felt this overloading is still
> worthwhile,
> > >> >> as
> > >> >>>> it
> > >> >>>>>> has
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>>>>>> following benefits:
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> > >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
> > >> >>> non-rich
> > >> >>>>>>>>>> functions
> > >> >>>>>>>>>>>>> to
> > >> >>>>>>>>>>>>>>> rich functions)
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed and I
> may
> > >> >>> have
> > >> >>>>>>>>>>>>> overlooked
> > >> >>>>>>>>>>>>>> in
> > >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> Guozhang
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > >> >>>>>>>>>>>>> matthias@confluent.io>
> > >> >>>>>>>>>>>>>>> wrote:
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> 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
> > >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
> > >> >>>> consisted
> > >> >>>>>>>>> API.
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> > >> >>>> overload.
> > >> >>>>>> It
> > >> >>>>>>>>>>>>> might
> > >> >>>>>>>>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> > >> >>>> overload
> > >> >>>>>>>>> that
> > >> >>>>>>>>>>>>> will
> > >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get
> too
> > >> >>> messy
> > >> >>>>>>>>> having
> > >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> > ValueMapperWithContext,
> > >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder pattern"
> > >> >> idea
> > >> >>>> as
> > >> >>>>> an
> > >> >>>>>>>>>> API
> > >> >>>>>>>>>>>>>>>> change and this might mitigate the overload problem.
> > Not
> > >> >>> for
> > >> >>>>>>>>> simple
> > >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> > >> >>>> aggregations.
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older email,
> I
> > >> >> am
> > >> >>>>>>>>>> personally
> > >> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and add
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> > >> >>>>>> `open(RecordContext)`
> > >> >>>>>>>>>> (or
> > >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not
> `close()`)
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> > >> >>>>> ValueMapper,
> > >> >>>>>>>>>>>>>>>> WithRecordContext
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of course,
> > we
> > >> >>>> don't
> > >> >>>>>>>>> get
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices
> Lambdas.
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the
> better
> > >> >>>> option
> > >> >>>>>>>>> might
> > >> >>>>>>>>>>>>> be.
> > >> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade
> off.
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> -Matthias
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > >> >>>>>>>>>>>>>>>>> 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.
> > >> >>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>> Cheers,
> > >> >>>>>>>>>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > >> >>>>>>>>> wangguoz@gmail.com>
> > >> >>>>>>>>>>>>>> wrote:
> > >> >>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>> 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 <
> > >> >>>>>>>>>>>>> je.karimov@gmail.com
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>> [1]
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> > >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> > >> >>>>>>>>>>>>>>>>>>> [2]
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>> Cheers,
> > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>>>>>>>>>> --
> > >> >>>>>>>>>>>>>>>>>>> -Cheers
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>>> --
> > >> >>>>>>>>>>>>>>>>>> -- Guozhang
> > >> >>>>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>>> --
> > >> >>>>>>>>>>>>> -Cheers
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>>>>
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> --
> > >> >>>>>>>>>> -Cheers
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Jeyhun
> > >> >>>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>> --
> > >> >>>>>>>>> -- Guozhang
> > >> >>>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>>> --
> > >> >>>>> -Cheers
> > >> >>>>>
> > >> >>>>> Jeyhun
> > >> >>>>>
> > >> >>>>
> > >> >>> --
> > >> >>> -Cheers
> > >> >>>
> > >> >>> Jeyhun
> > >> >>>
> > >> >>
> > >>
> > >> --
> > > -Cheers
> > >
> > > Jeyhun
> > >
> > --
> > -Cheers
> >
> > Jeyhun
> >
>
-- 
-Cheers

Jeyhun

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

Posted by Damian Guy <da...@gmail.com>.
Hi Jeyhun,

Thanks for sending out the update. I guess i was thinking more along the
lines of option 2 where we collapse the RichXXXX and ValueXXXXWithKey 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?

Thanks,
Damian

On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov <je...@gmail.com> wrote:

> 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 free to comment on this.
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=73637757
>
> Cheers,
> Jeyhun
>
> On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <je...@gmail.com>
> wrote:
>
> > 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 <ma...@confluent.io>
> > wrote:
> >
> >> 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
> >> > for one-shot solution for overloads issue.
> >> >
> >> > Cheers,
> >> > Jeyhun
> >> >
> >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com>
> >> wrote:
> >> >
> >> >> 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
> overrides.
> >> >>
> >> >> Thanks,
> >> >> Damian
> >> >>
> >> >>
> >> >>> About ProcessorContext vs RecordContext, you are right. I think I
> >> need to
> >> >>> implement a prototype to understand the full picture as some parts
> of
> >> the
> >> >>> KIP might not be as straightforward as I thought.
> >> >>>
> >> >>>
> >> >>> Cheers,
> >> >>> Jeyhun
> >> >>>
> >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com>
> >> wrote:
> >> >>>
> >> >>>> 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 lifetime of the
> >> >>>> Processor. Sot it doesn't make sense to cast a ProcessorContext to
> a
> >> >>>> RecordContext.
> >> >>>> You mentioned above passing the InternalProcessorContext to the
> >> init()
> >> >>>> calls. It is internal for a reason and i think it should remain
> that
> >> >> way.
> >> >>>> It might be better to move the recordContext() method from
> >> >>>> InternalProcessorContext to ProcessorContext.
> >> >>>>
> >> >>>> In the KIP you have an example showing:
> >> >>>> richMapper.init((RecordContext) processorContext);
> >> >>>> But the interface is:
> >> >>>> public interface RichValueMapper<V, VR> {
> >> >>>>     VR apply(final V value, final RecordContext recordContext);
> >> >>>> }
> >> >>>> i.e., there is no init(...), besides as above this wouldn't make
> >> sense.
> >> >>>>
> >> >>>> Thanks,
> >> >>>> Damian
> >> >>>>
> >> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com>
> >> >> wrote:
> >> >>>>
> >> >>>>> 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. Can't we pass this
> as a
> >> >>>>> parameter to init() method of processors? Then we would be able to
> >> >> get
> >> >>>>> RecordContext easily with just a method call.
> >> >>>>>
> >> >>>>>
> >> >>>>> Cheers,
> >> >>>>> Jeyhun
> >> >>>>>
> >> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> >> >>> matthias@confluent.io>
> >> >>>>> wrote:
> >> >>>>>
> >> >>>>>> 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
> >> >>> will
> >> >>>>>> be provided by the consecutive call to aggregate anyways.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> -Matthias
> >> >>>>>>
> >> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> >> >>>>>>> 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 implement both. This allows us
> >> >> to
> >> >>>> cast
> >> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> -Matthias
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> >> >>>>>>>> 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 address this issue as well or
> >> >>>>>>>> continue as it is?
> >> >>>>>>>>
> >> >>>>>>>> Cheers,
> >> >>>>>>>> Jeyhun
> >> >>>>>>>>
> >> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> >> >>> wangguoz@gmail.com>
> >> >>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>> LGTM. Thanks!
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> Guozhang
> >> >>>>>>>>>
> >> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> >> >>>>> je.karimov@gmail.com>
> >> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> >> >> integrated
> >> >>>> into
> >> >>>>>> the
> >> >>>>>>>>>> KIP by adding related method inside RecordContext interface.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Cheers,
> >> >>>>>>>>>> Jeyhun
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> >> >>>>>> matthias@confluent.io>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>> 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 information only. If you want
> >> >> to
> >> >>>> do
> >> >>>>> a
> >> >>>>>>>>>>> stateful computation you should rather use #transform().
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch to a
> >> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> For those reason, I advocate for a simple interface with a
> >> >>> single
> >> >>>>>>>>> method
> >> >>>>>>>>>>> that passes in a RecordContext object.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> -Matthias
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> >> >>>>>>>>>>>> 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 <
> >> >>>>>> je.karimov@gmail.com
> >> >>>>>>>>>>
> >> >>>>>>>>>>> wrote:
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>> 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, so I will not mention them.
> >> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in this
> >> >> KIP
> >> >>>>>>>>>> (KIP-159)
> >> >>>>>>>>>>>>> discussion threads.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
> >> >>> KIP-149,
> >> >>>>> in
> >> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
> >> >> support
> >> >>>>> Rich
> >> >>>>>>>>>>>>> functions as well.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
> >> >>>>>>>>>>> (ProcessorContext)
> >> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should not
> >> >>>> provide
> >> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated the
> >> >> two
> >> >>>>>>>>> problems
> >> >>>>>>>>>>> into
> >> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
> >> >>> parallel.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> - One approach we considered was :
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> >> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> >> >>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> >> >>>> RichFunction{
> >> >>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> public interface RichFunction {
> >> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> >> >>>>>>>>>>>>>     void close();
> >> >>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> public interface RecordContext {
> >> >>>>>>>>>>>>>     String applicationId();
> >> >>>>>>>>>>>>>     TaskId taskId();
> >> >>>>>>>>>>>>>     StreamsMetrics metrics();
> >> >>>>>>>>>>>>>     String topic();
> >> >>>>>>>>>>>>>     int partition();
> >> >>>>>>>>>>>>>     long offset();
> >> >>>>>>>>>>>>>     long timestamp();
> >> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> >> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> >> >> prefix);
> >> >>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> public interface ProcessorContext extends RecordContext {
> >> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> >> >>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> As a result:
> >> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> >> >>> converted
> >> >>>> to
> >> >>>>>>>>>> their
> >> >>>>>>>>>>>>> Rich counterparts (with empty init() and close() methods)
> >> >>>>>>>>>>>>> *. All related Processors will accept Rich interfaces in
> >> >>> their
> >> >>>>>>>>>>>>> constructors.
> >> >>>>>>>>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> >> >>>>> interfaces
> >> >>>>>>>>> to
> >> >>>>>>>>>>> Rich
> >> >>>>>>>>>>>>> interface while building the topology and initialize the
> >> >>>> related
> >> >>>>>>>>>>> processors
> >> >>>>>>>>>>>>> with Rich interfaces only.
> >> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> >> >> functions
> >> >>> as
> >> >>>>>> Rich
> >> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just check
> >> >> the
> >> >>>>> object
> >> >>>>>>>>>> type
> >> >>>>>>>>>>>>> and act accordingly.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> 3. There was some thoughts that the above approach does
> not
> >> >>>>> support
> >> >>>>>>>>>>> lambdas
> >> >>>>>>>>>>>>> so we should support only one method, only
> >> >>> init(RecordContext),
> >> >>>>> as
> >> >>>>>>>>>> part
> >> >>>>>>>>>>> of
> >> >>>>>>>>>>>>> Rich interfaces.
> >> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
> >> >>> interfaces
> >> >>>>> are
> >> >>>>>>>>> by
> >> >>>>>>>>>>>>> definition lambda-free and we should not care much about
> >> >> it.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> >> >>>> considered
> >> >>>>>> was
> >> >>>>>>>>>> to
> >> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This might
> >> >>> even
> >> >>>>>>>>> allow
> >> >>>>>>>>>> to
> >> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as we
> >> >>>>> preserve
> >> >>>>>>>>> the
> >> >>>>>>>>>>>>> nature of being a function.
> >> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
> >> >>>>>>>>>>>>> allow users to have an in-memory state for a function.
> >> >> Thus,
> >> >>> we
> >> >>>>>>>>> cannot
> >> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
> >> >>> multiple
> >> >>>>>> tasks
> >> >>>>>>>>>> and
> >> >>>>>>>>>>>>> we would need a supplier pattern similar to #transform().
> >> >> And
> >> >>>>> this
> >> >>>>>>>>>> would
> >> >>>>>>>>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
> >> >>> would
> >> >>>>> not
> >> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many new
> >> >>>> overload
> >> >>>>>>>>> for
> >> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's
> email)
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Cheers,
> >> >>>>>>>>>>>>> Jeyhun
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> >> >>>>>>>>> matthias@confluent.io
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>>> wrote:
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>> 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 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:
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> ---------------------------
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext context);
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> }
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> ...
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> // then in KStreams
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> >> >>>> extends
> >> >>>>>> VR>
> >> >>>>>>>>>>>>>> mapper);
> >> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper
> >> >> <?
> >> >>>>> super
> >> >>>>>>>>>> V, ?
> >> >>>>>>>>>>>>>>> extends VR> mapper);
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> -------------------------------
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> The caveat is that it will introduces more overloads;
> >> >> but I
> >> >>>>> think
> >> >>>>>>>>>> the
> >> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde overrides
> >> >> and
> >> >>>> 2)
> >> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
> >> >> reduced
> >> >>>> in
> >> >>>>>> the
> >> >>>>>>>>>>> near
> >> >>>>>>>>>>>>>>> future, and I felt this overloading is still worthwhile,
> >> >> as
> >> >>>> it
> >> >>>>>> has
> >> >>>>>>>>>> the
> >> >>>>>>>>>>>>>>> following benefits:
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> >> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
> >> >>> non-rich
> >> >>>>>>>>>> functions
> >> >>>>>>>>>>>>> to
> >> >>>>>>>>>>>>>>> rich functions)
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> Maybe this approach has already been discussed and I may
> >> >>> have
> >> >>>>>>>>>>>>> overlooked
> >> >>>>>>>>>>>>>> in
> >> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> Guozhang
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> >> >>>>>>>>>>>>> matthias@confluent.io>
> >> >>>>>>>>>>>>>>> wrote:
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> 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
> >> >>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
> >> >>>> consisted
> >> >>>>>>>>> API.
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> >> >>>> overload.
> >> >>>>>> It
> >> >>>>>>>>>>>>> might
> >> >>>>>>>>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> >> >>>> overload
> >> >>>>>>>>> that
> >> >>>>>>>>>>>>> will
> >> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get too
> >> >>> messy
> >> >>>>>>>>> having
> >> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey,
> ValueMapperWithContext,
> >> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder pattern"
> >> >> idea
> >> >>>> as
> >> >>>>> an
> >> >>>>>>>>>> API
> >> >>>>>>>>>>>>>>>> change and this might mitigate the overload problem.
> Not
> >> >>> for
> >> >>>>>>>>> simple
> >> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> >> >>>> aggregations.
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older email, I
> >> >> am
> >> >>>>>>>>>> personally
> >> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and add
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> >> >>>>>> `open(RecordContext)`
> >> >>>>>>>>>> (or
> >> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> >> >>>>> ValueMapper,
> >> >>>>>>>>>>>>>>>> WithRecordContext
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of course,
> we
> >> >>>> don't
> >> >>>>>>>>> get
> >> >>>>>>>>>> a
> >> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the better
> >> >>>> option
> >> >>>>>>>>> might
> >> >>>>>>>>>>>>> be.
> >> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade off.
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> -Matthias
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> >> >>>>>>>>>>>>>>>>> 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.
> >> >>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>> Cheers,
> >> >>>>>>>>>>>>>>>>> Jeyhun
> >> >>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> >> >>>>>>>>> wangguoz@gmail.com>
> >> >>>>>>>>>>>>>> wrote:
> >> >>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>> 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 <
> >> >>>>>>>>>>>>> je.karimov@gmail.com
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>> [1]
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> >> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> >> >>>>>>>>>>>>>>>>>>> [2]
> >> >>>>>>>>>>>>>>>>>>>
> >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>> Cheers,
> >> >>>>>>>>>>>>>>>>>>> Jeyhun
> >> >>>>>>>>>>>>>>>>>>> --
> >> >>>>>>>>>>>>>>>>>>> -Cheers
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>> Jeyhun
> >> >>>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>>> --
> >> >>>>>>>>>>>>>>>>>> -- Guozhang
> >> >>>>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>> --
> >> >>>>>>>>>>>>> -Cheers
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Jeyhun
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> --
> >> >>>>>>>>>> -Cheers
> >> >>>>>>>>>>
> >> >>>>>>>>>> Jeyhun
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> --
> >> >>>>>>>>> -- Guozhang
> >> >>>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>> --
> >> >>>>> -Cheers
> >> >>>>>
> >> >>>>> Jeyhun
> >> >>>>>
> >> >>>>
> >> >>> --
> >> >>> -Cheers
> >> >>>
> >> >>> Jeyhun
> >> >>>
> >> >>
> >>
> >> --
> > -Cheers
> >
> > Jeyhun
> >
> --
> -Cheers
>
> Jeyhun
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 free to comment on this.

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=73637757

Cheers,
Jeyhun

On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov <je...@gmail.com> wrote:

> 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 <ma...@confluent.io>
> wrote:
>
>> 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
>> > for one-shot solution for overloads issue.
>> >
>> > Cheers,
>> > Jeyhun
>> >
>> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com>
>> wrote:
>> >
>> >> 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 overrides.
>> >>
>> >> Thanks,
>> >> Damian
>> >>
>> >>
>> >>> About ProcessorContext vs RecordContext, you are right. I think I
>> need to
>> >>> implement a prototype to understand the full picture as some parts of
>> the
>> >>> KIP might not be as straightforward as I thought.
>> >>>
>> >>>
>> >>> Cheers,
>> >>> Jeyhun
>> >>>
>> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com>
>> wrote:
>> >>>
>> >>>> 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 lifetime of the
>> >>>> Processor. Sot it doesn't make sense to cast a ProcessorContext to a
>> >>>> RecordContext.
>> >>>> You mentioned above passing the InternalProcessorContext to the
>> init()
>> >>>> calls. It is internal for a reason and i think it should remain that
>> >> way.
>> >>>> It might be better to move the recordContext() method from
>> >>>> InternalProcessorContext to ProcessorContext.
>> >>>>
>> >>>> In the KIP you have an example showing:
>> >>>> richMapper.init((RecordContext) processorContext);
>> >>>> But the interface is:
>> >>>> public interface RichValueMapper<V, VR> {
>> >>>>     VR apply(final V value, final RecordContext recordContext);
>> >>>> }
>> >>>> i.e., there is no init(...), besides as above this wouldn't make
>> sense.
>> >>>>
>> >>>> Thanks,
>> >>>> Damian
>> >>>>
>> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> 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. Can't we pass this as a
>> >>>>> parameter to init() method of processors? Then we would be able to
>> >> get
>> >>>>> RecordContext easily with just a method call.
>> >>>>>
>> >>>>>
>> >>>>> Cheers,
>> >>>>> Jeyhun
>> >>>>>
>> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
>> >>> matthias@confluent.io>
>> >>>>> wrote:
>> >>>>>
>> >>>>>> 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
>> >>> will
>> >>>>>> be provided by the consecutive call to aggregate anyways.
>> >>>>>>
>> >>>>>>
>> >>>>>> -Matthias
>> >>>>>>
>> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
>> >>>>>>> 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 implement both. This allows us
>> >> to
>> >>>> cast
>> >>>>>>> it to `RecordContext` and thus limit the visible scope.
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> -Matthias
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
>> >>>>>>>> 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 address this issue as well or
>> >>>>>>>> continue as it is?
>> >>>>>>>>
>> >>>>>>>> Cheers,
>> >>>>>>>> Jeyhun
>> >>>>>>>>
>> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
>> >>> wangguoz@gmail.com>
>> >>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> LGTM. Thanks!
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> Guozhang
>> >>>>>>>>>
>> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
>> >>>>> je.karimov@gmail.com>
>> >>>>>>>>> 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 KAFKA-3907 [1] can also be
>> >> integrated
>> >>>> into
>> >>>>>> the
>> >>>>>>>>>> KIP by adding related method inside RecordContext interface.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Cheers,
>> >>>>>>>>>> Jeyhun
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
>> >>>>>> matthias@confluent.io>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>> 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 information only. If you want
>> >> to
>> >>>> do
>> >>>>> a
>> >>>>>>>>>>> stateful computation you should rather use #transform().
>> >>>>>>>>>>>
>> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch to a
>> >>>>>>>>>>> supplier-pattern introducing many more overloads.
>> >>>>>>>>>>>
>> >>>>>>>>>>> For those reason, I advocate for a simple interface with a
>> >>> single
>> >>>>>>>>> method
>> >>>>>>>>>>> that passes in a RecordContext object.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> -Matthias
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>> >>>>>>>>>>>> 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 <
>> >>>>>> je.karimov@gmail.com
>> >>>>>>>>>>
>> >>>>>>>>>>> wrote:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>> 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, so I will not mention them.
>> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in this
>> >> KIP
>> >>>>>>>>>> (KIP-159)
>> >>>>>>>>>>>>> discussion threads.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
>> >>> KIP-149,
>> >>>>> in
>> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
>> >> support
>> >>>>> Rich
>> >>>>>>>>>>>>> functions as well.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
>> >>>>>>>>>>> (ProcessorContext)
>> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should not
>> >>>> provide
>> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated the
>> >> two
>> >>>>>>>>> problems
>> >>>>>>>>>>> into
>> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
>> >>> parallel.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> - One approach we considered was :
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
>> >>>>>>>>>>>>>     VR apply(final K key, final V value);
>> >>>>>>>>>>>>> }
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
>> >>>> RichFunction{
>> >>>>>>>>>>>>> }
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> public interface RichFunction {
>> >>>>>>>>>>>>>     void init(RecordContext recordContext);
>> >>>>>>>>>>>>>     void close();
>> >>>>>>>>>>>>> }
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> public interface RecordContext {
>> >>>>>>>>>>>>>     String applicationId();
>> >>>>>>>>>>>>>     TaskId taskId();
>> >>>>>>>>>>>>>     StreamsMetrics metrics();
>> >>>>>>>>>>>>>     String topic();
>> >>>>>>>>>>>>>     int partition();
>> >>>>>>>>>>>>>     long offset();
>> >>>>>>>>>>>>>     long timestamp();
>> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
>> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
>> >> prefix);
>> >>>>>>>>>>>>> }
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> public interface ProcessorContext extends RecordContext {
>> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
>> >>>>>>>>>>>>> }
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> As a result:
>> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
>> >>> converted
>> >>>> to
>> >>>>>>>>>> their
>> >>>>>>>>>>>>> Rich counterparts (with empty init() and close() methods)
>> >>>>>>>>>>>>> *. All related Processors will accept Rich interfaces in
>> >>> their
>> >>>>>>>>>>>>> constructors.
>> >>>>>>>>>>>>> *. So, we convert the related "withKey" or "withoutKey"
>> >>>>> interfaces
>> >>>>>>>>> to
>> >>>>>>>>>>> Rich
>> >>>>>>>>>>>>> interface while building the topology and initialize the
>> >>>> related
>> >>>>>>>>>>> processors
>> >>>>>>>>>>>>> with Rich interfaces only.
>> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
>> >> functions
>> >>> as
>> >>>>>> Rich
>> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just check
>> >> the
>> >>>>> object
>> >>>>>>>>>> type
>> >>>>>>>>>>>>> and act accordingly.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> 3. There was some thoughts that the above approach does not
>> >>>>> support
>> >>>>>>>>>>> lambdas
>> >>>>>>>>>>>>> so we should support only one method, only
>> >>> init(RecordContext),
>> >>>>> as
>> >>>>>>>>>> part
>> >>>>>>>>>>> of
>> >>>>>>>>>>>>> Rich interfaces.
>> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
>> >>> interfaces
>> >>>>> are
>> >>>>>>>>> by
>> >>>>>>>>>>>>> definition lambda-free and we should not care much about
>> >> it.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
>> >>>> considered
>> >>>>>> was
>> >>>>>>>>>> to
>> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This might
>> >>> even
>> >>>>>>>>> allow
>> >>>>>>>>>> to
>> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as we
>> >>>>> preserve
>> >>>>>>>>> the
>> >>>>>>>>>>>>> nature of being a function.
>> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
>> >>>>>>>>>>>>> allow users to have an in-memory state for a function.
>> >> Thus,
>> >>> we
>> >>>>>>>>> cannot
>> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
>> >>> multiple
>> >>>>>> tasks
>> >>>>>>>>>> and
>> >>>>>>>>>>>>> we would need a supplier pattern similar to #transform().
>> >> And
>> >>>>> this
>> >>>>>>>>>> would
>> >>>>>>>>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
>> >>> would
>> >>>>> not
>> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many new
>> >>>> overload
>> >>>>>>>>> for
>> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Cheers,
>> >>>>>>>>>>>>> Jeyhun
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
>> >>>>>>>>> matthias@confluent.io
>> >>>>>>>>>>>
>> >>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>> 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 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:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> ---------------------------
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext context);
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> }
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> ...
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> // then in KStreams
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
>> >>>> extends
>> >>>>>> VR>
>> >>>>>>>>>>>>>> mapper);
>> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper
>> >> <?
>> >>>>> super
>> >>>>>>>>>> V, ?
>> >>>>>>>>>>>>>>> extends VR> mapper);
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> -------------------------------
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> The caveat is that it will introduces more overloads;
>> >> but I
>> >>>>> think
>> >>>>>>>>>> the
>> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde overrides
>> >> and
>> >>>> 2)
>> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
>> >> reduced
>> >>>> in
>> >>>>>> the
>> >>>>>>>>>>> near
>> >>>>>>>>>>>>>>> future, and I felt this overloading is still worthwhile,
>> >> as
>> >>>> it
>> >>>>>> has
>> >>>>>>>>>> the
>> >>>>>>>>>>>>>>> following benefits:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
>> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
>> >>> non-rich
>> >>>>>>>>>> functions
>> >>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>> rich functions)
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Maybe this approach has already been discussed and I may
>> >>> have
>> >>>>>>>>>>>>> overlooked
>> >>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Guozhang
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>> >>>>>>>>>>>>> matthias@confluent.io>
>> >>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> 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
>> >>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
>> >>>> consisted
>> >>>>>>>>> API.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
>> >>>> overload.
>> >>>>>> It
>> >>>>>>>>>>>>> might
>> >>>>>>>>>>>>>>>> be sufficient though, to do a single xxxWithContext()
>> >>>> overload
>> >>>>>>>>> that
>> >>>>>>>>>>>>> will
>> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get too
>> >>> messy
>> >>>>>>>>> having
>> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder pattern"
>> >> idea
>> >>>> as
>> >>>>> an
>> >>>>>>>>>> API
>> >>>>>>>>>>>>>>>> change and this might mitigate the overload problem. Not
>> >>> for
>> >>>>>>>>> simple
>> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
>> >>>> aggregations.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older email, I
>> >> am
>> >>>>>>>>>> personally
>> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and add
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
>> >>>>>> `open(RecordContext)`
>> >>>>>>>>>> (or
>> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
>> >>>>> ValueMapper,
>> >>>>>>>>>>>>>>>> WithRecordContext
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of course, we
>> >>>> don't
>> >>>>>>>>> get
>> >>>>>>>>>> a
>> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the better
>> >>>> option
>> >>>>>>>>> might
>> >>>>>>>>>>>>> be.
>> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade off.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> -Matthias
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>> >>>>>>>>>>>>>>>>> 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.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Cheers,
>> >>>>>>>>>>>>>>>>> Jeyhun
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
>> >>>>>>>>> wangguoz@gmail.com>
>> >>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> 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 <
>> >>>>>>>>>>>>> je.karimov@gmail.com
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> [1]
>> >>>>>>>>>>>>>>>>>>>
>> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
>> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
>> >>>>>>>>>>>>>>>>>>> [2]
>> >>>>>>>>>>>>>>>>>>>
>> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Cheers,
>> >>>>>>>>>>>>>>>>>>> Jeyhun
>> >>>>>>>>>>>>>>>>>>> --
>> >>>>>>>>>>>>>>>>>>> -Cheers
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Jeyhun
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> --
>> >>>>>>>>>>>>>>>>>> -- Guozhang
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> --
>> >>>>>>>>>>>>> -Cheers
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Jeyhun
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> --
>> >>>>>>>>>> -Cheers
>> >>>>>>>>>>
>> >>>>>>>>>> Jeyhun
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> --
>> >>>>>>>>> -- Guozhang
>> >>>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>> -Cheers
>> >>>>>
>> >>>>> Jeyhun
>> >>>>>
>> >>>>
>> >>> --
>> >>> -Cheers
>> >>>
>> >>> Jeyhun
>> >>>
>> >>
>>
>> --
> -Cheers
>
> Jeyhun
>
-- 
-Cheers

Jeyhun

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 <ma...@confluent.io>
wrote:

> 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
> > for one-shot solution for overloads issue.
> >
> > Cheers,
> > Jeyhun
> >
> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com> wrote:
> >
> >> 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 overrides.
> >>
> >> Thanks,
> >> Damian
> >>
> >>
> >>> About ProcessorContext vs RecordContext, you are right. I think I need
> to
> >>> implement a prototype to understand the full picture as some parts of
> the
> >>> KIP might not be as straightforward as I thought.
> >>>
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com>
> wrote:
> >>>
> >>>> 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 lifetime of the
> >>>> Processor. Sot it doesn't make sense to cast a ProcessorContext to a
> >>>> RecordContext.
> >>>> You mentioned above passing the InternalProcessorContext to the init()
> >>>> calls. It is internal for a reason and i think it should remain that
> >> way.
> >>>> It might be better to move the recordContext() method from
> >>>> InternalProcessorContext to ProcessorContext.
> >>>>
> >>>> In the KIP you have an example showing:
> >>>> richMapper.init((RecordContext) processorContext);
> >>>> But the interface is:
> >>>> public interface RichValueMapper<V, VR> {
> >>>>     VR apply(final V value, final RecordContext recordContext);
> >>>> }
> >>>> i.e., there is no init(...), besides as above this wouldn't make
> sense.
> >>>>
> >>>> Thanks,
> >>>> Damian
> >>>>
> >>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com>
> >> wrote:
> >>>>
> >>>>> 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. Can't we pass this as a
> >>>>> parameter to init() method of processors? Then we would be able to
> >> get
> >>>>> RecordContext easily with just a method call.
> >>>>>
> >>>>>
> >>>>> Cheers,
> >>>>> Jeyhun
> >>>>>
> >>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> >>> matthias@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> 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
> >>> will
> >>>>>> be provided by the consecutive call to aggregate anyways.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> >>>>>>> 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 implement both. This allows us
> >> to
> >>>> cast
> >>>>>>> it to `RecordContext` and thus limit the visible scope.
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> >>>>>>>> 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 address this issue as well or
> >>>>>>>> continue as it is?
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Jeyhun
> >>>>>>>>
> >>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> >>> wangguoz@gmail.com>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> LGTM. Thanks!
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> >>>>> je.karimov@gmail.com>
> >>>>>>>>> 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 KAFKA-3907 [1] can also be
> >> integrated
> >>>> into
> >>>>>> the
> >>>>>>>>>> KIP by adding related method inside RecordContext interface.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Jeyhun
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> >>>>>> matthias@confluent.io>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> 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 information only. If you want
> >> to
> >>>> do
> >>>>> a
> >>>>>>>>>>> stateful computation you should rather use #transform().
> >>>>>>>>>>>
> >>>>>>>>>>> Furthermore, as pointed out, we would need to switch to a
> >>>>>>>>>>> supplier-pattern introducing many more overloads.
> >>>>>>>>>>>
> >>>>>>>>>>> For those reason, I advocate for a simple interface with a
> >>> single
> >>>>>>>>> method
> >>>>>>>>>>> that passes in a RecordContext object.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> >>>>>>>>>>>> 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 <
> >>>>>> je.karimov@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 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, so I will not mention them.
> >>>>>>>>>>>>> The related discussions are spread in KIP-149 and in this
> >> KIP
> >>>>>>>>>> (KIP-159)
> >>>>>>>>>>>>> discussion threads.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
> >>> KIP-149,
> >>>>> in
> >>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
> >> support
> >>>>> Rich
> >>>>>>>>>>>>> functions as well.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
> >>>>>>>>>>> (ProcessorContext)
> >>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should not
> >>>> provide
> >>>>>>>>>>>>> ProcessorContext to users. As a result, we separated the
> >> two
> >>>>>>>>> problems
> >>>>>>>>>>> into
> >>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
> >>> parallel.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - One approach we considered was :
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> >>>>>>>>>>>>>     VR apply(final K key, final V value);
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
> >>>> RichFunction{
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> public interface RichFunction {
> >>>>>>>>>>>>>     void init(RecordContext recordContext);
> >>>>>>>>>>>>>     void close();
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> public interface RecordContext {
> >>>>>>>>>>>>>     String applicationId();
> >>>>>>>>>>>>>     TaskId taskId();
> >>>>>>>>>>>>>     StreamsMetrics metrics();
> >>>>>>>>>>>>>     String topic();
> >>>>>>>>>>>>>     int partition();
> >>>>>>>>>>>>>     long offset();
> >>>>>>>>>>>>>     long timestamp();
> >>>>>>>>>>>>>     Map<String, Object> appConfigs();
> >>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> >> prefix);
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> public interface ProcessorContext extends RecordContext {
> >>>>>>>>>>>>>    // all methods but the ones in RecordContext
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> As a result:
> >>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> >>> converted
> >>>> to
> >>>>>>>>>> their
> >>>>>>>>>>>>> Rich counterparts (with empty init() and close() methods)
> >>>>>>>>>>>>> *. All related Processors will accept Rich interfaces in
> >>> their
> >>>>>>>>>>>>> constructors.
> >>>>>>>>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> >>>>> interfaces
> >>>>>>>>> to
> >>>>>>>>>>> Rich
> >>>>>>>>>>>>> interface while building the topology and initialize the
> >>>> related
> >>>>>>>>>>> processors
> >>>>>>>>>>>>> with Rich interfaces only.
> >>>>>>>>>>>>> *. We will not need to overloaded methods for rich
> >> functions
> >>> as
> >>>>>> Rich
> >>>>>>>>>>>>> interfaces extend withKey interfaces. We will just check
> >> the
> >>>>> object
> >>>>>>>>>> type
> >>>>>>>>>>>>> and act accordingly.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 3. There was some thoughts that the above approach does not
> >>>>> support
> >>>>>>>>>>> lambdas
> >>>>>>>>>>>>> so we should support only one method, only
> >>> init(RecordContext),
> >>>>> as
> >>>>>>>>>> part
> >>>>>>>>>>> of
> >>>>>>>>>>>>> Rich interfaces.
> >>>>>>>>>>>>> This is still in discussion. Personally I think Rich
> >>> interfaces
> >>>>> are
> >>>>>>>>> by
> >>>>>>>>>>>>> definition lambda-free and we should not care much about
> >> it.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> >>>> considered
> >>>>>> was
> >>>>>>>>>> to
> >>>>>>>>>>>>> pass in the RecordContext as method parameter.  This might
> >>> even
> >>>>>>>>> allow
> >>>>>>>>>> to
> >>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as we
> >>>>> preserve
> >>>>>>>>> the
> >>>>>>>>>>>>> nature of being a function.
> >>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
> >>>>>>>>>>>>> allow users to have an in-memory state for a function.
> >> Thus,
> >>> we
> >>>>>>>>> cannot
> >>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
> >>> multiple
> >>>>>> tasks
> >>>>>>>>>> and
> >>>>>>>>>>>>> we would need a supplier pattern similar to #transform().
> >> And
> >>>>> this
> >>>>>>>>>> would
> >>>>>>>>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
> >>> would
> >>>>> not
> >>>>>>>>>>>>> inherit from ValueMapper and thus we would need many new
> >>>> overload
> >>>>>>>>> for
> >>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> >>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> 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 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:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> ---------------------------
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> VR apply(final V value, final RecordContext context);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> ...
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // then in KStreams
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> >>>> extends
> >>>>>> VR>
> >>>>>>>>>>>>>> mapper);
> >>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper
> >> <?
> >>>>> super
> >>>>>>>>>> V, ?
> >>>>>>>>>>>>>>> extends VR> mapper);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -------------------------------
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The caveat is that it will introduces more overloads;
> >> but I
> >>>>> think
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde overrides
> >> and
> >>>> 2)
> >>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
> >> reduced
> >>>> in
> >>>>>> the
> >>>>>>>>>>> near
> >>>>>>>>>>>>>>> future, and I felt this overloading is still worthwhile,
> >> as
> >>>> it
> >>>>>> has
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> following benefits:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1) still allow lambda expressions.
> >>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
> >>> non-rich
> >>>>>>>>>> functions
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> rich functions)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Maybe this approach has already been discussed and I may
> >>> have
> >>>>>>>>>>>>> overlooked
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>> the email thread; anyways, lmk.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> >>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
> >>>> consisted
> >>>>>>>>> API.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> >>>> overload.
> >>>>>> It
> >>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> >>>> overload
> >>>>>>>>> that
> >>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get too
> >>> messy
> >>>>>>>>> having
> >>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> >>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On the other hand, we also have the "builder pattern"
> >> idea
> >>>> as
> >>>>> an
> >>>>>>>>>> API
> >>>>>>>>>>>>>>>> change and this might mitigate the overload problem. Not
> >>> for
> >>>>>>>>> simple
> >>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
> >>>> aggregations.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older email, I
> >> am
> >>>>>>>>>> personally
> >>>>>>>>>>>>>>>> fine to break the pure functional interface, and add
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>   - interface WithRecordContext with method
> >>>>>> `open(RecordContext)`
> >>>>>>>>>> (or
> >>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> >>>>> ValueMapper,
> >>>>>>>>>>>>>>>> WithRecordContext
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of course, we
> >>>> don't
> >>>>>>>>> get
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I am personally a little bit undecided what the better
> >>>> option
> >>>>>>>>> might
> >>>>>>>>>>>>> be.
> >>>>>>>>>>>>>>>> Curious to hear what other think about this trade off.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> >>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>> je.karimov@gmail.com
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 like to get your comments.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> [1]
> >>>>>>>>>>>>>>>>>>>
> >>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
> >>>>>>>>>>>>>>>>>>> and+ValueJoiner
> >>>>>>>>>>>>>>>>>>> [2]
> >>>>>>>>>>>>>>>>>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>> -Cheers
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> --
> >>>>>>>>>>>>> -Cheers
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> --
> >>>>>>>>>> -Cheers
> >>>>>>>>>>
> >>>>>>>>>> Jeyhun
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> --
> >>>>>>>>> -- Guozhang
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>> --
> >>>>> -Cheers
> >>>>>
> >>>>> Jeyhun
> >>>>>
> >>>>
> >>> --
> >>> -Cheers
> >>>
> >>> Jeyhun
> >>>
> >>
>
> --
-Cheers

Jeyhun

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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
> for one-shot solution for overloads issue.
> 
> Cheers,
> Jeyhun
> 
> On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <da...@gmail.com> wrote:
> 
>> 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 overrides.
>>
>> Thanks,
>> Damian
>>
>>
>>> About ProcessorContext vs RecordContext, you are right. I think I need to
>>> implement a prototype to understand the full picture as some parts of the
>>> KIP might not be as straightforward as I thought.
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com> wrote:
>>>
>>>> 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 lifetime of the
>>>> Processor. Sot it doesn't make sense to cast a ProcessorContext to a
>>>> RecordContext.
>>>> You mentioned above passing the InternalProcessorContext to the init()
>>>> calls. It is internal for a reason and i think it should remain that
>> way.
>>>> It might be better to move the recordContext() method from
>>>> InternalProcessorContext to ProcessorContext.
>>>>
>>>> In the KIP you have an example showing:
>>>> richMapper.init((RecordContext) processorContext);
>>>> But the interface is:
>>>> public interface RichValueMapper<V, VR> {
>>>>     VR apply(final V value, final RecordContext recordContext);
>>>> }
>>>> i.e., there is no init(...), besides as above this wouldn't make sense.
>>>>
>>>> Thanks,
>>>> Damian
>>>>
>>>> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com>
>> wrote:
>>>>
>>>>> 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. Can't we pass this as a
>>>>> parameter to init() method of processors? Then we would be able to
>> get
>>>>> RecordContext easily with just a method call.
>>>>>
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
>>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> 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
>>> will
>>>>>> be provided by the consecutive call to aggregate anyways.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
>>>>>>> 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 implement both. This allows us
>> to
>>>> cast
>>>>>>> it to `RecordContext` and thus limit the visible scope.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
>>>>>>>> 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 address this issue as well or
>>>>>>>> continue as it is?
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
>>> wangguoz@gmail.com>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> LGTM. Thanks!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
>>>>> je.karimov@gmail.com>
>>>>>>>>> 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 KAFKA-3907 [1] can also be
>> integrated
>>>> into
>>>>>> the
>>>>>>>>>> KIP by adding related method inside RecordContext interface.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
>>>>>> matthias@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 information only. If you want
>> to
>>>> do
>>>>> a
>>>>>>>>>>> stateful computation you should rather use #transform().
>>>>>>>>>>>
>>>>>>>>>>> Furthermore, as pointed out, we would need to switch to a
>>>>>>>>>>> supplier-pattern introducing many more overloads.
>>>>>>>>>>>
>>>>>>>>>>> For those reason, I advocate for a simple interface with a
>>> single
>>>>>>>>> method
>>>>>>>>>>> that passes in a RecordContext object.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>>>>>>>>>>>> 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 <
>>>>>> je.karimov@gmail.com
>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> 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, so I will not mention them.
>>>>>>>>>>>>> The related discussions are spread in KIP-149 and in this
>> KIP
>>>>>>>>>> (KIP-159)
>>>>>>>>>>>>> discussion threads.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1. The idea of rich functions came into the stage with
>>> KIP-149,
>>>>> in
>>>>>>>>>>>>> discussion thread. As a result we extended KIP-149 to
>> support
>>>>> Rich
>>>>>>>>>>>>> functions as well.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2.  To as part of the Rich functions, we provided init
>>>>>>>>>>> (ProcessorContext)
>>>>>>>>>>>>> method. Afterwards, Dammian suggested that we should not
>>>> provide
>>>>>>>>>>>>> ProcessorContext to users. As a result, we separated the
>> two
>>>>>>>>> problems
>>>>>>>>>>> into
>>>>>>>>>>>>> two separate KIPs, as it seems they can be solved in
>>> parallel.
>>>>>>>>>>>>>
>>>>>>>>>>>>> - One approach we considered was :
>>>>>>>>>>>>>
>>>>>>>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
>>>>>>>>>>>>>     VR apply(final K key, final V value);
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> public interface RichValueMapper<K, V, VR> extends
>>>> RichFunction{
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> public interface RichFunction {
>>>>>>>>>>>>>     void init(RecordContext recordContext);
>>>>>>>>>>>>>     void close();
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> public interface RecordContext {
>>>>>>>>>>>>>     String applicationId();
>>>>>>>>>>>>>     TaskId taskId();
>>>>>>>>>>>>>     StreamsMetrics metrics();
>>>>>>>>>>>>>     String topic();
>>>>>>>>>>>>>     int partition();
>>>>>>>>>>>>>     long offset();
>>>>>>>>>>>>>     long timestamp();
>>>>>>>>>>>>>     Map<String, Object> appConfigs();
>>>>>>>>>>>>>     Map<String, Object> appConfigsWithPrefix(String
>> prefix);
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> public interface ProcessorContext extends RecordContext {
>>>>>>>>>>>>>    // all methods but the ones in RecordContext
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> As a result:
>>>>>>>>>>>>> * . All "withKey" and "withoutKey" interfaces can be
>>> converted
>>>> to
>>>>>>>>>> their
>>>>>>>>>>>>> Rich counterparts (with empty init() and close() methods)
>>>>>>>>>>>>> *. All related Processors will accept Rich interfaces in
>>> their
>>>>>>>>>>>>> constructors.
>>>>>>>>>>>>> *. So, we convert the related "withKey" or "withoutKey"
>>>>> interfaces
>>>>>>>>> to
>>>>>>>>>>> Rich
>>>>>>>>>>>>> interface while building the topology and initialize the
>>>> related
>>>>>>>>>>> processors
>>>>>>>>>>>>> with Rich interfaces only.
>>>>>>>>>>>>> *. We will not need to overloaded methods for rich
>> functions
>>> as
>>>>>> Rich
>>>>>>>>>>>>> interfaces extend withKey interfaces. We will just check
>> the
>>>>> object
>>>>>>>>>> type
>>>>>>>>>>>>> and act accordingly.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3. There was some thoughts that the above approach does not
>>>>> support
>>>>>>>>>>> lambdas
>>>>>>>>>>>>> so we should support only one method, only
>>> init(RecordContext),
>>>>> as
>>>>>>>>>> part
>>>>>>>>>>> of
>>>>>>>>>>>>> Rich interfaces.
>>>>>>>>>>>>> This is still in discussion. Personally I think Rich
>>> interfaces
>>>>> are
>>>>>>>>> by
>>>>>>>>>>>>> definition lambda-free and we should not care much about
>> it.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 4. Thanks to Matthias's discussion, an alternative we
>>>> considered
>>>>>> was
>>>>>>>>>> to
>>>>>>>>>>>>> pass in the RecordContext as method parameter.  This might
>>> even
>>>>>>>>> allow
>>>>>>>>>> to
>>>>>>>>>>>>> use Lambdas and we could keep the name RichFunction as we
>>>>> preserve
>>>>>>>>> the
>>>>>>>>>>>>> nature of being a function.
>>>>>>>>>>>>> "If you go with `init()` and `close()` we basically
>>>>>>>>>>>>> allow users to have an in-memory state for a function.
>> Thus,
>>> we
>>>>>>>>> cannot
>>>>>>>>>>>>> share a single instance of RichValueMapper (etc) over
>>> multiple
>>>>>> tasks
>>>>>>>>>> and
>>>>>>>>>>>>> we would need a supplier pattern similar to #transform().
>> And
>>>>> this
>>>>>>>>>> would
>>>>>>>>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
>>> would
>>>>> not
>>>>>>>>>>>>> inherit from ValueMapper and thus we would need many new
>>>> overload
>>>>>>>>> for
>>>>>>>>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 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:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ---------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> interface RichValueMapper<V, VR> {
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> VR apply(final V value, final RecordContext context);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ...
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // then in KStreams
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
>>>> extends
>>>>>> VR>
>>>>>>>>>>>>>> mapper);
>>>>>>>>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper
>> <?
>>>>> super
>>>>>>>>>> V, ?
>>>>>>>>>>>>>>> extends VR> mapper);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -------------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The caveat is that it will introduces more overloads;
>> but I
>>>>> think
>>>>>>>>>> the
>>>>>>>>>>>>>>> #.overloads are mainly introduced by 1) serde overrides
>> and
>>>> 2)
>>>>>>>>>>>>>>> state-store-supplier overides, both of which can be
>> reduced
>>>> in
>>>>>> the
>>>>>>>>>>> near
>>>>>>>>>>>>>>> future, and I felt this overloading is still worthwhile,
>> as
>>>> it
>>>>>> has
>>>>>>>>>> the
>>>>>>>>>>>>>>> following benefits:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1) still allow lambda expressions.
>>>>>>>>>>>>>>> 2) clearer code path (do not need to "convert" from
>>> non-rich
>>>>>>>>>> functions
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> rich functions)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Maybe this approach has already been discussed and I may
>>> have
>>>>>>>>>>>>> overlooked
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the email thread; anyways, lmk.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>> to get an overall picture to converge to an overall
>>>> consisted
>>>>>>>>> API.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
>>>> overload.
>>>>>> It
>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>> be sufficient though, to do a single xxxWithContext()
>>>> overload
>>>>>>>>> that
>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> provide key+value+context. Otherwise, if might get too
>>> messy
>>>>>>>>> having
>>>>>>>>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>>>>>>>>>>>>>>>> ValueMapperWithKeyWithContext.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On the other hand, we also have the "builder pattern"
>> idea
>>>> as
>>>>> an
>>>>>>>>>> API
>>>>>>>>>>>>>>>> change and this might mitigate the overload problem. Not
>>> for
>>>>>>>>> simple
>>>>>>>>>>>>>>>> function like map/flatMap etc but for joins and
>>>> aggregations.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On the other hand, as I mentioned in an older email, I
>> am
>>>>>>>>>> personally
>>>>>>>>>>>>>>>> fine to break the pure functional interface, and add
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>   - interface WithRecordContext with method
>>>>>> `open(RecordContext)`
>>>>>>>>>> (or
>>>>>>>>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>   - interface ValueMapperWithRecordContext extends
>>>>> ValueMapper,
>>>>>>>>>>>>>>>> WithRecordContext
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This would allow us to avoid any overload. Of course, we
>>>> don't
>>>>>>>>> get
>>>>>>>>>> a
>>>>>>>>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I am personally a little bit undecided what the better
>>>> option
>>>>>>>>> might
>>>>>>>>>>>>> be.
>>>>>>>>>>>>>>>> Curious to hear what other think about this trade off.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>> je.karimov@gmail.com
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 like to get your comments.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>
>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>>>>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>>>>>>>>>>>>>>>> ValueTransformer+ValueMapper+
>>>>>>>>>>>>>>>>>>> and+ValueJoiner
>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>> -Cheers
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>>>>>>>> -Cheers
>>>>>>>>>>>>>
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>> -Cheers
>>>>>>>>>>
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> -- Guozhang
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>> --
>>>>> -Cheers
>>>>>
>>>>> Jeyhun
>>>>>
>>>>
>>> --
>>> -Cheers
>>>
>>> Jeyhun
>>>
>>


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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 <da...@gmail.com> wrote:

> 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 overrides.
>
> Thanks,
> Damian
>
>
> > About ProcessorContext vs RecordContext, you are right. I think I need to
> > implement a prototype to understand the full picture as some parts of the
> > KIP might not be as straightforward as I thought.
> >
> >
> > Cheers,
> > Jeyhun
> >
> > On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com> wrote:
> >
> > > 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 lifetime of the
> > > Processor. Sot it doesn't make sense to cast a ProcessorContext to a
> > > RecordContext.
> > > You mentioned above passing the InternalProcessorContext to the init()
> > > calls. It is internal for a reason and i think it should remain that
> way.
> > > It might be better to move the recordContext() method from
> > > InternalProcessorContext to ProcessorContext.
> > >
> > > In the KIP you have an example showing:
> > > richMapper.init((RecordContext) processorContext);
> > > But the interface is:
> > > public interface RichValueMapper<V, VR> {
> > >     VR apply(final V value, final RecordContext recordContext);
> > > }
> > > i.e., there is no init(...), besides as above this wouldn't make sense.
> > >
> > > Thanks,
> > > Damian
> > >
> > > On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com>
> wrote:
> > >
> > > > 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. Can't we pass this as a
> > > > parameter to init() method of processors? Then we would be able to
> get
> > > > RecordContext easily with just a method call.
> > > >
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> > matthias@confluent.io>
> > > > wrote:
> > > >
> > > > > 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
> > will
> > > > > be provided by the consecutive call to aggregate anyways.
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > > On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > > 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 implement both. This allows us
> to
> > > cast
> > > > > > it to `RecordContext` and thus limit the visible scope.
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > >
> > > > > >
> > > > > > On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > > >> 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 address this issue as well or
> > > > > >> continue as it is?
> > > > > >>
> > > > > >> Cheers,
> > > > > >> Jeyhun
> > > > > >>
> > > > > >> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > wrote:
> > > > > >>
> > > > > >>> LGTM. Thanks!
> > > > > >>>
> > > > > >>>
> > > > > >>> Guozhang
> > > > > >>>
> > > > > >>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > > je.karimov@gmail.com>
> > > > > >>> 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 KAFKA-3907 [1] can also be
> integrated
> > > into
> > > > > the
> > > > > >>>> KIP by adding related method inside RecordContext interface.
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> Cheers,
> > > > > >>>> Jeyhun
> > > > > >>>>
> > > > > >>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > > matthias@confluent.io>
> > > > > >>>> wrote:
> > > > > >>>>
> > > > > >>>>> 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 information only. If you want
> to
> > > do
> > > > a
> > > > > >>>>> stateful computation you should rather use #transform().
> > > > > >>>>>
> > > > > >>>>> Furthermore, as pointed out, we would need to switch to a
> > > > > >>>>> supplier-pattern introducing many more overloads.
> > > > > >>>>>
> > > > > >>>>> For those reason, I advocate for a simple interface with a
> > single
> > > > > >>> method
> > > > > >>>>> that passes in a RecordContext object.
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>> -Matthias
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > > >>>>>> 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 <
> > > > > je.karimov@gmail.com
> > > > > >>>>
> > > > > >>>>> wrote:
> > > > > >>>>>>
> > > > > >>>>>>> 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, so I will not mention them.
> > > > > >>>>>>> The related discussions are spread in KIP-149 and in this
> KIP
> > > > > >>>> (KIP-159)
> > > > > >>>>>>> discussion threads.
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> 1. The idea of rich functions came into the stage with
> > KIP-149,
> > > > in
> > > > > >>>>>>> discussion thread. As a result we extended KIP-149 to
> support
> > > > Rich
> > > > > >>>>>>> functions as well.
> > > > > >>>>>>>
> > > > > >>>>>>> 2.  To as part of the Rich functions, we provided init
> > > > > >>>>> (ProcessorContext)
> > > > > >>>>>>> method. Afterwards, Dammian suggested that we should not
> > > provide
> > > > > >>>>>>> ProcessorContext to users. As a result, we separated the
> two
> > > > > >>> problems
> > > > > >>>>> into
> > > > > >>>>>>> two separate KIPs, as it seems they can be solved in
> > parallel.
> > > > > >>>>>>>
> > > > > >>>>>>> - One approach we considered was :
> > > > > >>>>>>>
> > > > > >>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > > >>>>>>>     VR apply(final K key, final V value);
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>> public interface RichValueMapper<K, V, VR> extends
> > > RichFunction{
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>> public interface RichFunction {
> > > > > >>>>>>>     void init(RecordContext recordContext);
> > > > > >>>>>>>     void close();
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>> public interface RecordContext {
> > > > > >>>>>>>     String applicationId();
> > > > > >>>>>>>     TaskId taskId();
> > > > > >>>>>>>     StreamsMetrics metrics();
> > > > > >>>>>>>     String topic();
> > > > > >>>>>>>     int partition();
> > > > > >>>>>>>     long offset();
> > > > > >>>>>>>     long timestamp();
> > > > > >>>>>>>     Map<String, Object> appConfigs();
> > > > > >>>>>>>     Map<String, Object> appConfigsWithPrefix(String
> prefix);
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> public interface ProcessorContext extends RecordContext {
> > > > > >>>>>>>    // all methods but the ones in RecordContext
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>> As a result:
> > > > > >>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> > converted
> > > to
> > > > > >>>> their
> > > > > >>>>>>> Rich counterparts (with empty init() and close() methods)
> > > > > >>>>>>> *. All related Processors will accept Rich interfaces in
> > their
> > > > > >>>>>>> constructors.
> > > > > >>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> > > > interfaces
> > > > > >>> to
> > > > > >>>>> Rich
> > > > > >>>>>>> interface while building the topology and initialize the
> > > related
> > > > > >>>>> processors
> > > > > >>>>>>> with Rich interfaces only.
> > > > > >>>>>>> *. We will not need to overloaded methods for rich
> functions
> > as
> > > > > Rich
> > > > > >>>>>>> interfaces extend withKey interfaces. We will just check
> the
> > > > object
> > > > > >>>> type
> > > > > >>>>>>> and act accordingly.
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> 3. There was some thoughts that the above approach does not
> > > > support
> > > > > >>>>> lambdas
> > > > > >>>>>>> so we should support only one method, only
> > init(RecordContext),
> > > > as
> > > > > >>>> part
> > > > > >>>>> of
> > > > > >>>>>>> Rich interfaces.
> > > > > >>>>>>> This is still in discussion. Personally I think Rich
> > interfaces
> > > > are
> > > > > >>> by
> > > > > >>>>>>> definition lambda-free and we should not care much about
> it.
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> > > considered
> > > > > was
> > > > > >>>> to
> > > > > >>>>>>> pass in the RecordContext as method parameter.  This might
> > even
> > > > > >>> allow
> > > > > >>>> to
> > > > > >>>>>>> use Lambdas and we could keep the name RichFunction as we
> > > > preserve
> > > > > >>> the
> > > > > >>>>>>> nature of being a function.
> > > > > >>>>>>> "If you go with `init()` and `close()` we basically
> > > > > >>>>>>> allow users to have an in-memory state for a function.
> Thus,
> > we
> > > > > >>> cannot
> > > > > >>>>>>> share a single instance of RichValueMapper (etc) over
> > multiple
> > > > > tasks
> > > > > >>>> and
> > > > > >>>>>>> we would need a supplier pattern similar to #transform().
> And
> > > > this
> > > > > >>>> would
> > > > > >>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
> > would
> > > > not
> > > > > >>>>>>> inherit from ValueMapper and thus we would need many new
> > > overload
> > > > > >>> for
> > > > > >>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> Cheers,
> > > > > >>>>>>> Jeyhun
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > > >>> matthias@confluent.io
> > > > > >>>>>
> > > > > >>>>>>> wrote:
> > > > > >>>>>>>
> > > > > >>>>>>>> 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 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:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> ---------------------------
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> interface RichValueMapper<V, VR> {
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> VR apply(final V value, final RecordContext context);
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> }
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> ...
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> // then in KStreams
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> > > extends
> > > > > VR>
> > > > > >>>>>>>> mapper);
> > > > > >>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper
> <?
> > > > super
> > > > > >>>> V, ?
> > > > > >>>>>>>>> extends VR> mapper);
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> -------------------------------
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> The caveat is that it will introduces more overloads;
> but I
> > > > think
> > > > > >>>> the
> > > > > >>>>>>>>> #.overloads are mainly introduced by 1) serde overrides
> and
> > > 2)
> > > > > >>>>>>>>> state-store-supplier overides, both of which can be
> reduced
> > > in
> > > > > the
> > > > > >>>>> near
> > > > > >>>>>>>>> future, and I felt this overloading is still worthwhile,
> as
> > > it
> > > > > has
> > > > > >>>> the
> > > > > >>>>>>>>> following benefits:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> 1) still allow lambda expressions.
> > > > > >>>>>>>>> 2) clearer code path (do not need to "convert" from
> > non-rich
> > > > > >>>> functions
> > > > > >>>>>>> to
> > > > > >>>>>>>>> rich functions)
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> Maybe this approach has already been discussed and I may
> > have
> > > > > >>>>>>> overlooked
> > > > > >>>>>>>> in
> > > > > >>>>>>>>> the email thread; anyways, lmk.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> Guozhang
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > > > >>>>>>> matthias@confluent.io>
> > > > > >>>>>>>>> wrote:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>> 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
> > > > > >>>>>>>>>> to get an overall picture to converge to an overall
> > > consisted
> > > > > >>> API.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> > > overload.
> > > > > It
> > > > > >>>>>>> might
> > > > > >>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> > > overload
> > > > > >>> that
> > > > > >>>>>>> will
> > > > > >>>>>>>>>> provide key+value+context. Otherwise, if might get too
> > messy
> > > > > >>> having
> > > > > >>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > > > > >>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On the other hand, we also have the "builder pattern"
> idea
> > > as
> > > > an
> > > > > >>>> API
> > > > > >>>>>>>>>> change and this might mitigate the overload problem. Not
> > for
> > > > > >>> simple
> > > > > >>>>>>>>>> function like map/flatMap etc but for joins and
> > > aggregations.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On the other hand, as I mentioned in an older email, I
> am
> > > > > >>>> personally
> > > > > >>>>>>>>>> fine to break the pure functional interface, and add
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>   - interface WithRecordContext with method
> > > > > `open(RecordContext)`
> > > > > >>>> (or
> > > > > >>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> > > > ValueMapper,
> > > > > >>>>>>>>>> WithRecordContext
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> This would allow us to avoid any overload. Of course, we
> > > don't
> > > > > >>> get
> > > > > >>>> a
> > > > > >>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> I am personally a little bit undecided what the better
> > > option
> > > > > >>> might
> > > > > >>>>>>> be.
> > > > > >>>>>>>>>> Curious to hear what other think about this trade off.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> -Matthias
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > > >>>>>>>>>>> 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.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Cheers,
> > > > > >>>>>>>>>>> Jeyhun
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > > > >>> wangguoz@gmail.com>
> > > > > >>>>>>>> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> 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 <
> > > > > >>>>>>> je.karimov@gmail.com
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>>>> 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 like to get your comments.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> [1]
> > > > > >>>>>>>>>>>>>
> > > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > > >>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > > >>>>>>>>>> ValueTransformer+ValueMapper+
> > > > > >>>>>>>>>>>>> and+ValueJoiner
> > > > > >>>>>>>>>>>>> [2]
> > > > > >>>>>>>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > >>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Cheers,
> > > > > >>>>>>>>>>>>> Jeyhun
> > > > > >>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>> -Cheers
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Jeyhun
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> --
> > > > > >>>>>>>>>>>> -- Guozhang
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>> --
> > > > > >>>>>>> -Cheers
> > > > > >>>>>>>
> > > > > >>>>>>> Jeyhun
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>> --
> > > > > >>>> -Cheers
> > > > > >>>>
> > > > > >>>> Jeyhun
> > > > > >>>>
> > > > > >>>
> > > > > >>>
> > > > > >>>
> > > > > >>> --
> > > > > >>> -- Guozhang
> > > > > >>>
> > > > > >
> > > > >
> > > > > --
> > > > -Cheers
> > > >
> > > > Jeyhun
> > > >
> > >
> > --
> > -Cheers
> >
> > Jeyhun
> >
>
-- 
-Cheers

Jeyhun

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

Posted by Damian Guy <da...@gmail.com>.
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 overrides.

Thanks,
Damian


> About ProcessorContext vs RecordContext, you are right. I think I need to
> implement a prototype to understand the full picture as some parts of the
> KIP might not be as straightforward as I thought.
>
>
> Cheers,
> Jeyhun
>
> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com> wrote:
>
> > 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 lifetime of the
> > Processor. Sot it doesn't make sense to cast a ProcessorContext to a
> > RecordContext.
> > You mentioned above passing the InternalProcessorContext to the init()
> > calls. It is internal for a reason and i think it should remain that way.
> > It might be better to move the recordContext() method from
> > InternalProcessorContext to ProcessorContext.
> >
> > In the KIP you have an example showing:
> > richMapper.init((RecordContext) processorContext);
> > But the interface is:
> > public interface RichValueMapper<V, VR> {
> >     VR apply(final V value, final RecordContext recordContext);
> > }
> > i.e., there is no init(...), besides as above this wouldn't make sense.
> >
> > Thanks,
> > Damian
> >
> > On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com> wrote:
> >
> > > 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. Can't we pass this as a
> > > parameter to init() method of processors? Then we would be able to get
> > > RecordContext easily with just a method call.
> > >
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <
> matthias@confluent.io>
> > > wrote:
> > >
> > > > 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
> will
> > > > be provided by the consecutive call to aggregate anyways.
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > > 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 implement both. This allows us to
> > cast
> > > > > it to `RecordContext` and thus limit the visible scope.
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > >
> > > > >
> > > > > On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > > >> 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 address this issue as well or
> > > > >> continue as it is?
> > > > >>
> > > > >> Cheers,
> > > > >> Jeyhun
> > > > >>
> > > > >> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > wrote:
> > > > >>
> > > > >>> LGTM. Thanks!
> > > > >>>
> > > > >>>
> > > > >>> Guozhang
> > > > >>>
> > > > >>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > > je.karimov@gmail.com>
> > > > >>> 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 KAFKA-3907 [1] can also be integrated
> > into
> > > > the
> > > > >>>> KIP by adding related method inside RecordContext interface.
> > > > >>>>
> > > > >>>>
> > > > >>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > > >>>>
> > > > >>>>
> > > > >>>> Cheers,
> > > > >>>> Jeyhun
> > > > >>>>
> > > > >>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > > matthias@confluent.io>
> > > > >>>> wrote:
> > > > >>>>
> > > > >>>>> 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 information only. If you want to
> > do
> > > a
> > > > >>>>> stateful computation you should rather use #transform().
> > > > >>>>>
> > > > >>>>> Furthermore, as pointed out, we would need to switch to a
> > > > >>>>> supplier-pattern introducing many more overloads.
> > > > >>>>>
> > > > >>>>> For those reason, I advocate for a simple interface with a
> single
> > > > >>> method
> > > > >>>>> that passes in a RecordContext object.
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> -Matthias
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > >>>>>> 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 <
> > > > je.karimov@gmail.com
> > > > >>>>
> > > > >>>>> wrote:
> > > > >>>>>>
> > > > >>>>>>> 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, so I will not mention them.
> > > > >>>>>>> The related discussions are spread in KIP-149 and in this KIP
> > > > >>>> (KIP-159)
> > > > >>>>>>> discussion threads.
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> 1. The idea of rich functions came into the stage with
> KIP-149,
> > > in
> > > > >>>>>>> discussion thread. As a result we extended KIP-149 to support
> > > Rich
> > > > >>>>>>> functions as well.
> > > > >>>>>>>
> > > > >>>>>>> 2.  To as part of the Rich functions, we provided init
> > > > >>>>> (ProcessorContext)
> > > > >>>>>>> method. Afterwards, Dammian suggested that we should not
> > provide
> > > > >>>>>>> ProcessorContext to users. As a result, we separated the two
> > > > >>> problems
> > > > >>>>> into
> > > > >>>>>>> two separate KIPs, as it seems they can be solved in
> parallel.
> > > > >>>>>>>
> > > > >>>>>>> - One approach we considered was :
> > > > >>>>>>>
> > > > >>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > > >>>>>>>     VR apply(final K key, final V value);
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>> public interface RichValueMapper<K, V, VR> extends
> > RichFunction{
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>> public interface RichFunction {
> > > > >>>>>>>     void init(RecordContext recordContext);
> > > > >>>>>>>     void close();
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>> public interface RecordContext {
> > > > >>>>>>>     String applicationId();
> > > > >>>>>>>     TaskId taskId();
> > > > >>>>>>>     StreamsMetrics metrics();
> > > > >>>>>>>     String topic();
> > > > >>>>>>>     int partition();
> > > > >>>>>>>     long offset();
> > > > >>>>>>>     long timestamp();
> > > > >>>>>>>     Map<String, Object> appConfigs();
> > > > >>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> public interface ProcessorContext extends RecordContext {
> > > > >>>>>>>    // all methods but the ones in RecordContext
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>> As a result:
> > > > >>>>>>> * . All "withKey" and "withoutKey" interfaces can be
> converted
> > to
> > > > >>>> their
> > > > >>>>>>> Rich counterparts (with empty init() and close() methods)
> > > > >>>>>>> *. All related Processors will accept Rich interfaces in
> their
> > > > >>>>>>> constructors.
> > > > >>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> > > interfaces
> > > > >>> to
> > > > >>>>> Rich
> > > > >>>>>>> interface while building the topology and initialize the
> > related
> > > > >>>>> processors
> > > > >>>>>>> with Rich interfaces only.
> > > > >>>>>>> *. We will not need to overloaded methods for rich functions
> as
> > > > Rich
> > > > >>>>>>> interfaces extend withKey interfaces. We will just check the
> > > object
> > > > >>>> type
> > > > >>>>>>> and act accordingly.
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> 3. There was some thoughts that the above approach does not
> > > support
> > > > >>>>> lambdas
> > > > >>>>>>> so we should support only one method, only
> init(RecordContext),
> > > as
> > > > >>>> part
> > > > >>>>> of
> > > > >>>>>>> Rich interfaces.
> > > > >>>>>>> This is still in discussion. Personally I think Rich
> interfaces
> > > are
> > > > >>> by
> > > > >>>>>>> definition lambda-free and we should not care much about it.
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> > considered
> > > > was
> > > > >>>> to
> > > > >>>>>>> pass in the RecordContext as method parameter.  This might
> even
> > > > >>> allow
> > > > >>>> to
> > > > >>>>>>> use Lambdas and we could keep the name RichFunction as we
> > > preserve
> > > > >>> the
> > > > >>>>>>> nature of being a function.
> > > > >>>>>>> "If you go with `init()` and `close()` we basically
> > > > >>>>>>> allow users to have an in-memory state for a function. Thus,
> we
> > > > >>> cannot
> > > > >>>>>>> share a single instance of RichValueMapper (etc) over
> multiple
> > > > tasks
> > > > >>>> and
> > > > >>>>>>> we would need a supplier pattern similar to #transform(). And
> > > this
> > > > >>>> would
> > > > >>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier
> would
> > > not
> > > > >>>>>>> inherit from ValueMapper and thus we would need many new
> > overload
> > > > >>> for
> > > > >>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> Cheers,
> > > > >>>>>>> Jeyhun
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > > >>> matthias@confluent.io
> > > > >>>>>
> > > > >>>>>>> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> 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 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:
> > > > >>>>>>>>>
> > > > >>>>>>>>> ---------------------------
> > > > >>>>>>>>>
> > > > >>>>>>>>> interface RichValueMapper<V, VR> {
> > > > >>>>>>>>>
> > > > >>>>>>>>> VR apply(final V value, final RecordContext context);
> > > > >>>>>>>>>
> > > > >>>>>>>>> }
> > > > >>>>>>>>>
> > > > >>>>>>>>> ...
> > > > >>>>>>>>>
> > > > >>>>>>>>> // then in KStreams
> > > > >>>>>>>>>
> > > > >>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> > extends
> > > > VR>
> > > > >>>>>>>> mapper);
> > > > >>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <?
> > > super
> > > > >>>> V, ?
> > > > >>>>>>>>> extends VR> mapper);
> > > > >>>>>>>>>
> > > > >>>>>>>>> -------------------------------
> > > > >>>>>>>>>
> > > > >>>>>>>>> The caveat is that it will introduces more overloads; but I
> > > think
> > > > >>>> the
> > > > >>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and
> > 2)
> > > > >>>>>>>>> state-store-supplier overides, both of which can be reduced
> > in
> > > > the
> > > > >>>>> near
> > > > >>>>>>>>> future, and I felt this overloading is still worthwhile, as
> > it
> > > > has
> > > > >>>> the
> > > > >>>>>>>>> following benefits:
> > > > >>>>>>>>>
> > > > >>>>>>>>> 1) still allow lambda expressions.
> > > > >>>>>>>>> 2) clearer code path (do not need to "convert" from
> non-rich
> > > > >>>> functions
> > > > >>>>>>> to
> > > > >>>>>>>>> rich functions)
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>> Maybe this approach has already been discussed and I may
> have
> > > > >>>>>>> overlooked
> > > > >>>>>>>> in
> > > > >>>>>>>>> the email thread; anyways, lmk.
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>> Guozhang
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > > >>>>>>> matthias@confluent.io>
> > > > >>>>>>>>> wrote:
> > > > >>>>>>>>>
> > > > >>>>>>>>>> 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
> > > > >>>>>>>>>> to get an overall picture to converge to an overall
> > consisted
> > > > >>> API.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> > overload.
> > > > It
> > > > >>>>>>> might
> > > > >>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> > overload
> > > > >>> that
> > > > >>>>>>> will
> > > > >>>>>>>>>> provide key+value+context. Otherwise, if might get too
> messy
> > > > >>> having
> > > > >>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > > > >>>>>>>>>> ValueMapperWithKeyWithContext.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On the other hand, we also have the "builder pattern" idea
> > as
> > > an
> > > > >>>> API
> > > > >>>>>>>>>> change and this might mitigate the overload problem. Not
> for
> > > > >>> simple
> > > > >>>>>>>>>> function like map/flatMap etc but for joins and
> > aggregations.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On the other hand, as I mentioned in an older email, I am
> > > > >>>> personally
> > > > >>>>>>>>>> fine to break the pure functional interface, and add
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>   - interface WithRecordContext with method
> > > > `open(RecordContext)`
> > > > >>>> (or
> > > > >>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> > > ValueMapper,
> > > > >>>>>>>>>> WithRecordContext
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> This would allow us to avoid any overload. Of course, we
> > don't
> > > > >>> get
> > > > >>>> a
> > > > >>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> I am personally a little bit undecided what the better
> > option
> > > > >>> might
> > > > >>>>>>> be.
> > > > >>>>>>>>>> Curious to hear what other think about this trade off.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> -Matthias
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > > >>>>>>>>>>> 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.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Cheers,
> > > > >>>>>>>>>>> Jeyhun
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > > >>> wangguoz@gmail.com>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> 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 <
> > > > >>>>>>> je.karimov@gmail.com
> > > > >>>>>>>>>
> > > > >>>>>>>>>>>> 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 like to get your comments.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> [1]
> > > > >>>>>>>>>>>>>
> > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > > >>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > > >>>>>>>>>> ValueTransformer+ValueMapper+
> > > > >>>>>>>>>>>>> and+ValueJoiner
> > > > >>>>>>>>>>>>> [2]
> > > > >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Cheers,
> > > > >>>>>>>>>>>>> Jeyhun
> > > > >>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>> -Cheers
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Jeyhun
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> --
> > > > >>>>>>>>>>>> -- Guozhang
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> --
> > > > >>>>>>> -Cheers
> > > > >>>>>>>
> > > > >>>>>>> Jeyhun
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>> --
> > > > >>>> -Cheers
> > > > >>>>
> > > > >>>> Jeyhun
> > > > >>>>
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>> --
> > > > >>> -- Guozhang
> > > > >>>
> > > > >
> > > >
> > > > --
> > > -Cheers
> > >
> > > Jeyhun
> > >
> >
> --
> -Cheers
>
> Jeyhun
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 some parts of the
KIP might not be as straightforward as I thought.


Cheers,
Jeyhun

On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <da...@gmail.com> wrote:

> 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 lifetime of the
> Processor. Sot it doesn't make sense to cast a ProcessorContext to a
> RecordContext.
> You mentioned above passing the InternalProcessorContext to the init()
> calls. It is internal for a reason and i think it should remain that way.
> It might be better to move the recordContext() method from
> InternalProcessorContext to ProcessorContext.
>
> In the KIP you have an example showing:
> richMapper.init((RecordContext) processorContext);
> But the interface is:
> public interface RichValueMapper<V, VR> {
>     VR apply(final V value, final RecordContext recordContext);
> }
> i.e., there is no init(...), besides as above this wouldn't make sense.
>
> Thanks,
> Damian
>
> On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com> wrote:
>
> > 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. Can't we pass this as a
> > parameter to init() method of processors? Then we would be able to get
> > RecordContext easily with just a method call.
> >
> >
> > Cheers,
> > Jeyhun
> >
> > On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > 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 will
> > > be provided by the consecutive call to aggregate anyways.
> > >
> > >
> > > -Matthias
> > >
> > > On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > > 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 implement both. This allows us to
> cast
> > > > it to `RecordContext` and thus limit the visible scope.
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > >
> > > > On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > > >> 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 address this issue as well or
> > > >> continue as it is?
> > > >>
> > > >> Cheers,
> > > >> Jeyhun
> > > >>
> > > >> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >>
> > > >>> LGTM. Thanks!
> > > >>>
> > > >>>
> > > >>> Guozhang
> > > >>>
> > > >>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> > je.karimov@gmail.com>
> > > >>> 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 KAFKA-3907 [1] can also be integrated
> into
> > > the
> > > >>>> KIP by adding related method inside RecordContext interface.
> > > >>>>
> > > >>>>
> > > >>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > > >>>>
> > > >>>>
> > > >>>> Cheers,
> > > >>>> Jeyhun
> > > >>>>
> > > >>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > > matthias@confluent.io>
> > > >>>> wrote:
> > > >>>>
> > > >>>>> 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 information only. If you want to
> do
> > a
> > > >>>>> stateful computation you should rather use #transform().
> > > >>>>>
> > > >>>>> Furthermore, as pointed out, we would need to switch to a
> > > >>>>> supplier-pattern introducing many more overloads.
> > > >>>>>
> > > >>>>> For those reason, I advocate for a simple interface with a single
> > > >>> method
> > > >>>>> that passes in a RecordContext object.
> > > >>>>>
> > > >>>>>
> > > >>>>> -Matthias
> > > >>>>>
> > > >>>>>
> > > >>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > >>>>>> 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 <
> > > je.karimov@gmail.com
> > > >>>>
> > > >>>>> wrote:
> > > >>>>>>
> > > >>>>>>> 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, so I will not mention them.
> > > >>>>>>> The related discussions are spread in KIP-149 and in this KIP
> > > >>>> (KIP-159)
> > > >>>>>>> discussion threads.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 1. The idea of rich functions came into the stage with KIP-149,
> > in
> > > >>>>>>> discussion thread. As a result we extended KIP-149 to support
> > Rich
> > > >>>>>>> functions as well.
> > > >>>>>>>
> > > >>>>>>> 2.  To as part of the Rich functions, we provided init
> > > >>>>> (ProcessorContext)
> > > >>>>>>> method. Afterwards, Dammian suggested that we should not
> provide
> > > >>>>>>> ProcessorContext to users. As a result, we separated the two
> > > >>> problems
> > > >>>>> into
> > > >>>>>>> two separate KIPs, as it seems they can be solved in parallel.
> > > >>>>>>>
> > > >>>>>>> - One approach we considered was :
> > > >>>>>>>
> > > >>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > > >>>>>>>     VR apply(final K key, final V value);
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>> public interface RichValueMapper<K, V, VR> extends
> RichFunction{
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>> public interface RichFunction {
> > > >>>>>>>     void init(RecordContext recordContext);
> > > >>>>>>>     void close();
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>> public interface RecordContext {
> > > >>>>>>>     String applicationId();
> > > >>>>>>>     TaskId taskId();
> > > >>>>>>>     StreamsMetrics metrics();
> > > >>>>>>>     String topic();
> > > >>>>>>>     int partition();
> > > >>>>>>>     long offset();
> > > >>>>>>>     long timestamp();
> > > >>>>>>>     Map<String, Object> appConfigs();
> > > >>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> public interface ProcessorContext extends RecordContext {
> > > >>>>>>>    // all methods but the ones in RecordContext
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>> As a result:
> > > >>>>>>> * . All "withKey" and "withoutKey" interfaces can be converted
> to
> > > >>>> their
> > > >>>>>>> Rich counterparts (with empty init() and close() methods)
> > > >>>>>>> *. All related Processors will accept Rich interfaces in their
> > > >>>>>>> constructors.
> > > >>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> > interfaces
> > > >>> to
> > > >>>>> Rich
> > > >>>>>>> interface while building the topology and initialize the
> related
> > > >>>>> processors
> > > >>>>>>> with Rich interfaces only.
> > > >>>>>>> *. We will not need to overloaded methods for rich functions as
> > > Rich
> > > >>>>>>> interfaces extend withKey interfaces. We will just check the
> > object
> > > >>>> type
> > > >>>>>>> and act accordingly.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 3. There was some thoughts that the above approach does not
> > support
> > > >>>>> lambdas
> > > >>>>>>> so we should support only one method, only init(RecordContext),
> > as
> > > >>>> part
> > > >>>>> of
> > > >>>>>>> Rich interfaces.
> > > >>>>>>> This is still in discussion. Personally I think Rich interfaces
> > are
> > > >>> by
> > > >>>>>>> definition lambda-free and we should not care much about it.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 4. Thanks to Matthias's discussion, an alternative we
> considered
> > > was
> > > >>>> to
> > > >>>>>>> pass in the RecordContext as method parameter.  This might even
> > > >>> allow
> > > >>>> to
> > > >>>>>>> use Lambdas and we could keep the name RichFunction as we
> > preserve
> > > >>> the
> > > >>>>>>> nature of being a function.
> > > >>>>>>> "If you go with `init()` and `close()` we basically
> > > >>>>>>> allow users to have an in-memory state for a function. Thus, we
> > > >>> cannot
> > > >>>>>>> share a single instance of RichValueMapper (etc) over multiple
> > > tasks
> > > >>>> and
> > > >>>>>>> we would need a supplier pattern similar to #transform(). And
> > this
> > > >>>> would
> > > >>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would
> > not
> > > >>>>>>> inherit from ValueMapper and thus we would need many new
> overload
> > > >>> for
> > > >>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> Cheers,
> > > >>>>>>> Jeyhun
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > > >>> matthias@confluent.io
> > > >>>>>
> > > >>>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>>> 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 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:
> > > >>>>>>>>>
> > > >>>>>>>>> ---------------------------
> > > >>>>>>>>>
> > > >>>>>>>>> interface RichValueMapper<V, VR> {
> > > >>>>>>>>>
> > > >>>>>>>>> VR apply(final V value, final RecordContext context);
> > > >>>>>>>>>
> > > >>>>>>>>> }
> > > >>>>>>>>>
> > > >>>>>>>>> ...
> > > >>>>>>>>>
> > > >>>>>>>>> // then in KStreams
> > > >>>>>>>>>
> > > >>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ?
> extends
> > > VR>
> > > >>>>>>>> mapper);
> > > >>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <?
> > super
> > > >>>> V, ?
> > > >>>>>>>>> extends VR> mapper);
> > > >>>>>>>>>
> > > >>>>>>>>> -------------------------------
> > > >>>>>>>>>
> > > >>>>>>>>> The caveat is that it will introduces more overloads; but I
> > think
> > > >>>> the
> > > >>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and
> 2)
> > > >>>>>>>>> state-store-supplier overides, both of which can be reduced
> in
> > > the
> > > >>>>> near
> > > >>>>>>>>> future, and I felt this overloading is still worthwhile, as
> it
> > > has
> > > >>>> the
> > > >>>>>>>>> following benefits:
> > > >>>>>>>>>
> > > >>>>>>>>> 1) still allow lambda expressions.
> > > >>>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
> > > >>>> functions
> > > >>>>>>> to
> > > >>>>>>>>> rich functions)
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> Maybe this approach has already been discussed and I may have
> > > >>>>>>> overlooked
> > > >>>>>>>> in
> > > >>>>>>>>> the email thread; anyways, lmk.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> Guozhang
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > >>>>>>> matthias@confluent.io>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> 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
> > > >>>>>>>>>> to get an overall picture to converge to an overall
> consisted
> > > >>> API.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more
> overload.
> > > It
> > > >>>>>>> might
> > > >>>>>>>>>> be sufficient though, to do a single xxxWithContext()
> overload
> > > >>> that
> > > >>>>>>> will
> > > >>>>>>>>>> provide key+value+context. Otherwise, if might get too messy
> > > >>> having
> > > >>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > > >>>>>>>>>> ValueMapperWithKeyWithContext.
> > > >>>>>>>>>>
> > > >>>>>>>>>> On the other hand, we also have the "builder pattern" idea
> as
> > an
> > > >>>> API
> > > >>>>>>>>>> change and this might mitigate the overload problem. Not for
> > > >>> simple
> > > >>>>>>>>>> function like map/flatMap etc but for joins and
> aggregations.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On the other hand, as I mentioned in an older email, I am
> > > >>>> personally
> > > >>>>>>>>>> fine to break the pure functional interface, and add
> > > >>>>>>>>>>
> > > >>>>>>>>>>   - interface WithRecordContext with method
> > > `open(RecordContext)`
> > > >>>> (or
> > > >>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> > > >>>>>>>>>>
> > > >>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> > ValueMapper,
> > > >>>>>>>>>> WithRecordContext
> > > >>>>>>>>>>
> > > >>>>>>>>>> This would allow us to avoid any overload. Of course, we
> don't
> > > >>> get
> > > >>>> a
> > > >>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> I am personally a little bit undecided what the better
> option
> > > >>> might
> > > >>>>>>> be.
> > > >>>>>>>>>> Curious to hear what other think about this trade off.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> -Matthias
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > >>>>>>>>>>> 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.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Cheers,
> > > >>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > > >>> wangguoz@gmail.com>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> 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 <
> > > >>>>>>> je.karimov@gmail.com
> > > >>>>>>>>>
> > > >>>>>>>>>>>> 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 like to get your comments.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> [1]
> > > >>>>>>>>>>>>>
> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > >>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > >>>>>>>>>> ValueTransformer+ValueMapper+
> > > >>>>>>>>>>>>> and+ValueJoiner
> > > >>>>>>>>>>>>> [2]
> > > >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>> --
> > > >>>>>>>>>>>>> -Cheers
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Jeyhun
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> --
> > > >>>>>>>>>>>> -- Guozhang
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> --
> > > >>>>>>> -Cheers
> > > >>>>>>>
> > > >>>>>>> Jeyhun
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>> --
> > > >>>> -Cheers
> > > >>>>
> > > >>>> Jeyhun
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>> --
> > > >>> -- Guozhang
> > > >>>
> > > >
> > >
> > > --
> > -Cheers
> >
> > Jeyhun
> >
>
-- 
-Cheers

Jeyhun

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

Posted by Damian Guy <da...@gmail.com>.
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 lifetime of the
Processor. Sot it doesn't make sense to cast a ProcessorContext to a
RecordContext.
You mentioned above passing the InternalProcessorContext to the init()
calls. It is internal for a reason and i think it should remain that way.
It might be better to move the recordContext() method from
InternalProcessorContext to ProcessorContext.

In the KIP you have an example showing:
richMapper.init((RecordContext) processorContext);
But the interface is:
public interface RichValueMapper<V, VR> {
    VR apply(final V value, final RecordContext recordContext);
}
i.e., there is no init(...), besides as above this wouldn't make sense.

Thanks,
Damian

On Tue, 4 Jul 2017 at 23:30 Jeyhun Karimov <je...@gmail.com> wrote:

> 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. Can't we pass this as a
> parameter to init() method of processors? Then we would be able to get
> RecordContext easily with just a method call.
>
>
> Cheers,
> Jeyhun
>
> On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > 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 will
> > be provided by the consecutive call to aggregate anyways.
> >
> >
> > -Matthias
> >
> > On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > > 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 implement both. This allows us to cast
> > > it to `RecordContext` and thus limit the visible scope.
> > >
> > >
> > > -Matthias
> > >
> > >
> > >
> > > On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> > >> 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 address this issue as well or
> > >> continue as it is?
> > >>
> > >> Cheers,
> > >> Jeyhun
> > >>
> > >> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>
> > >>> LGTM. Thanks!
> > >>>
> > >>>
> > >>> Guozhang
> > >>>
> > >>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <
> je.karimov@gmail.com>
> > >>> 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 KAFKA-3907 [1] can also be integrated into
> > the
> > >>>> KIP by adding related method inside RecordContext interface.
> > >>>>
> > >>>>
> > >>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> > >>>>
> > >>>>
> > >>>> Cheers,
> > >>>> Jeyhun
> > >>>>
> > >>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> > matthias@confluent.io>
> > >>>> wrote:
> > >>>>
> > >>>>> 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 information only. If you want to do
> a
> > >>>>> stateful computation you should rather use #transform().
> > >>>>>
> > >>>>> Furthermore, as pointed out, we would need to switch to a
> > >>>>> supplier-pattern introducing many more overloads.
> > >>>>>
> > >>>>> For those reason, I advocate for a simple interface with a single
> > >>> method
> > >>>>> that passes in a RecordContext object.
> > >>>>>
> > >>>>>
> > >>>>> -Matthias
> > >>>>>
> > >>>>>
> > >>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > >>>>>> 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 <
> > je.karimov@gmail.com
> > >>>>
> > >>>>> wrote:
> > >>>>>>
> > >>>>>>> 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, so I will not mention them.
> > >>>>>>> The related discussions are spread in KIP-149 and in this KIP
> > >>>> (KIP-159)
> > >>>>>>> discussion threads.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 1. The idea of rich functions came into the stage with KIP-149,
> in
> > >>>>>>> discussion thread. As a result we extended KIP-149 to support
> Rich
> > >>>>>>> functions as well.
> > >>>>>>>
> > >>>>>>> 2.  To as part of the Rich functions, we provided init
> > >>>>> (ProcessorContext)
> > >>>>>>> method. Afterwards, Dammian suggested that we should not provide
> > >>>>>>> ProcessorContext to users. As a result, we separated the two
> > >>> problems
> > >>>>> into
> > >>>>>>> two separate KIPs, as it seems they can be solved in parallel.
> > >>>>>>>
> > >>>>>>> - One approach we considered was :
> > >>>>>>>
> > >>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> > >>>>>>>     VR apply(final K key, final V value);
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>> public interface RichValueMapper<K, V, VR> extends RichFunction{
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>> public interface RichFunction {
> > >>>>>>>     void init(RecordContext recordContext);
> > >>>>>>>     void close();
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>> public interface RecordContext {
> > >>>>>>>     String applicationId();
> > >>>>>>>     TaskId taskId();
> > >>>>>>>     StreamsMetrics metrics();
> > >>>>>>>     String topic();
> > >>>>>>>     int partition();
> > >>>>>>>     long offset();
> > >>>>>>>     long timestamp();
> > >>>>>>>     Map<String, Object> appConfigs();
> > >>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> public interface ProcessorContext extends RecordContext {
> > >>>>>>>    // all methods but the ones in RecordContext
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>> As a result:
> > >>>>>>> * . All "withKey" and "withoutKey" interfaces can be converted to
> > >>>> their
> > >>>>>>> Rich counterparts (with empty init() and close() methods)
> > >>>>>>> *. All related Processors will accept Rich interfaces in their
> > >>>>>>> constructors.
> > >>>>>>> *. So, we convert the related "withKey" or "withoutKey"
> interfaces
> > >>> to
> > >>>>> Rich
> > >>>>>>> interface while building the topology and initialize the related
> > >>>>> processors
> > >>>>>>> with Rich interfaces only.
> > >>>>>>> *. We will not need to overloaded methods for rich functions as
> > Rich
> > >>>>>>> interfaces extend withKey interfaces. We will just check the
> object
> > >>>> type
> > >>>>>>> and act accordingly.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 3. There was some thoughts that the above approach does not
> support
> > >>>>> lambdas
> > >>>>>>> so we should support only one method, only init(RecordContext),
> as
> > >>>> part
> > >>>>> of
> > >>>>>>> Rich interfaces.
> > >>>>>>> This is still in discussion. Personally I think Rich interfaces
> are
> > >>> by
> > >>>>>>> definition lambda-free and we should not care much about it.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 4. Thanks to Matthias's discussion, an alternative we considered
> > was
> > >>>> to
> > >>>>>>> pass in the RecordContext as method parameter.  This might even
> > >>> allow
> > >>>> to
> > >>>>>>> use Lambdas and we could keep the name RichFunction as we
> preserve
> > >>> the
> > >>>>>>> nature of being a function.
> > >>>>>>> "If you go with `init()` and `close()` we basically
> > >>>>>>> allow users to have an in-memory state for a function. Thus, we
> > >>> cannot
> > >>>>>>> share a single instance of RichValueMapper (etc) over multiple
> > tasks
> > >>>> and
> > >>>>>>> we would need a supplier pattern similar to #transform(). And
> this
> > >>>> would
> > >>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would
> not
> > >>>>>>> inherit from ValueMapper and thus we would need many new overload
> > >>> for
> > >>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> Cheers,
> > >>>>>>> Jeyhun
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> > >>> matthias@confluent.io
> > >>>>>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> 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 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:
> > >>>>>>>>>
> > >>>>>>>>> ---------------------------
> > >>>>>>>>>
> > >>>>>>>>> interface RichValueMapper<V, VR> {
> > >>>>>>>>>
> > >>>>>>>>> VR apply(final V value, final RecordContext context);
> > >>>>>>>>>
> > >>>>>>>>> }
> > >>>>>>>>>
> > >>>>>>>>> ...
> > >>>>>>>>>
> > >>>>>>>>> // then in KStreams
> > >>>>>>>>>
> > >>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends
> > VR>
> > >>>>>>>> mapper);
> > >>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <?
> super
> > >>>> V, ?
> > >>>>>>>>> extends VR> mapper);
> > >>>>>>>>>
> > >>>>>>>>> -------------------------------
> > >>>>>>>>>
> > >>>>>>>>> The caveat is that it will introduces more overloads; but I
> think
> > >>>> the
> > >>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
> > >>>>>>>>> state-store-supplier overides, both of which can be reduced in
> > the
> > >>>>> near
> > >>>>>>>>> future, and I felt this overloading is still worthwhile, as it
> > has
> > >>>> the
> > >>>>>>>>> following benefits:
> > >>>>>>>>>
> > >>>>>>>>> 1) still allow lambda expressions.
> > >>>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
> > >>>> functions
> > >>>>>>> to
> > >>>>>>>>> rich functions)
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> Maybe this approach has already been discussed and I may have
> > >>>>>>> overlooked
> > >>>>>>>> in
> > >>>>>>>>> the email thread; anyways, lmk.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> Guozhang
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > >>>>>>> matthias@confluent.io>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> 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
> > >>>>>>>>>> to get an overall picture to converge to an overall consisted
> > >>> API.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more overload.
> > It
> > >>>>>>> might
> > >>>>>>>>>> be sufficient though, to do a single xxxWithContext() overload
> > >>> that
> > >>>>>>> will
> > >>>>>>>>>> provide key+value+context. Otherwise, if might get too messy
> > >>> having
> > >>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > >>>>>>>>>> ValueMapperWithKeyWithContext.
> > >>>>>>>>>>
> > >>>>>>>>>> On the other hand, we also have the "builder pattern" idea as
> an
> > >>>> API
> > >>>>>>>>>> change and this might mitigate the overload problem. Not for
> > >>> simple
> > >>>>>>>>>> function like map/flatMap etc but for joins and aggregations.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On the other hand, as I mentioned in an older email, I am
> > >>>> personally
> > >>>>>>>>>> fine to break the pure functional interface, and add
> > >>>>>>>>>>
> > >>>>>>>>>>   - interface WithRecordContext with method
> > `open(RecordContext)`
> > >>>> (or
> > >>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> > >>>>>>>>>>
> > >>>>>>>>>>   - interface ValueMapperWithRecordContext extends
> ValueMapper,
> > >>>>>>>>>> WithRecordContext
> > >>>>>>>>>>
> > >>>>>>>>>> This would allow us to avoid any overload. Of course, we don't
> > >>> get
> > >>>> a
> > >>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> I am personally a little bit undecided what the better option
> > >>> might
> > >>>>>>> be.
> > >>>>>>>>>> Curious to hear what other think about this trade off.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Matthias
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > >>>>>>>>>>> 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.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Cheers,
> > >>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> > >>> wangguoz@gmail.com>
> > >>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> 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 <
> > >>>>>>> je.karimov@gmail.com
> > >>>>>>>>>
> > >>>>>>>>>>>> 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 like to get your comments.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > >>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > >>>>>>>>>> ValueTransformer+ValueMapper+
> > >>>>>>>>>>>>> and+ValueJoiner
> > >>>>>>>>>>>>> [2]
> > >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>> --
> > >>>>>>>>>>>>> -Cheers
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> --
> > >>>>>>>>>>>> -- Guozhang
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>> --
> > >>>>>>> -Cheers
> > >>>>>>>
> > >>>>>>> Jeyhun
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>> --
> > >>>> -Cheers
> > >>>>
> > >>>> Jeyhun
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>> --
> > >>> -- Guozhang
> > >>>
> > >
> >
> > --
> -Cheers
>
> Jeyhun
>

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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. Can't we pass this as a
parameter to init() method of processors? Then we would be able to get
RecordContext easily with just a method call.


Cheers,
Jeyhun

On Thu, Jun 29, 2017 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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 will
> be provided by the consecutive call to aggregate anyways.
>
>
> -Matthias
>
> On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> > 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 implement both. This allows us to cast
> > it to `RecordContext` and thus limit the visible scope.
> >
> >
> > -Matthias
> >
> >
> >
> > On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> >> 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 address this issue as well or
> >> continue as it is?
> >>
> >> Cheers,
> >> Jeyhun
> >>
> >> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> LGTM. Thanks!
> >>>
> >>>
> >>> Guozhang
> >>>
> >>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <je...@gmail.com>
> >>> 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 KAFKA-3907 [1] can also be integrated into
> the
> >>>> KIP by adding related method inside RecordContext interface.
> >>>>
> >>>>
> >>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >>>>
> >>>>
> >>>> Cheers,
> >>>> Jeyhun
> >>>>
> >>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <
> matthias@confluent.io>
> >>>> wrote:
> >>>>
> >>>>> 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 information only. If you want to do a
> >>>>> stateful computation you should rather use #transform().
> >>>>>
> >>>>> Furthermore, as pointed out, we would need to switch to a
> >>>>> supplier-pattern introducing many more overloads.
> >>>>>
> >>>>> For those reason, I advocate for a simple interface with a single
> >>> method
> >>>>> that passes in a RecordContext object.
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> >>>>>> 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 <
> je.karimov@gmail.com
> >>>>
> >>>>> wrote:
> >>>>>>
> >>>>>>> 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, so I will not mention them.
> >>>>>>> The related discussions are spread in KIP-149 and in this KIP
> >>>> (KIP-159)
> >>>>>>> discussion threads.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 1. The idea of rich functions came into the stage with KIP-149, in
> >>>>>>> discussion thread. As a result we extended KIP-149 to support Rich
> >>>>>>> functions as well.
> >>>>>>>
> >>>>>>> 2.  To as part of the Rich functions, we provided init
> >>>>> (ProcessorContext)
> >>>>>>> method. Afterwards, Dammian suggested that we should not provide
> >>>>>>> ProcessorContext to users. As a result, we separated the two
> >>> problems
> >>>>> into
> >>>>>>> two separate KIPs, as it seems they can be solved in parallel.
> >>>>>>>
> >>>>>>> - One approach we considered was :
> >>>>>>>
> >>>>>>> public interface ValueMapperWithKey<K, V, VR> {
> >>>>>>>     VR apply(final K key, final V value);
> >>>>>>> }
> >>>>>>>
> >>>>>>> public interface RichValueMapper<K, V, VR> extends RichFunction{
> >>>>>>> }
> >>>>>>>
> >>>>>>> public interface RichFunction {
> >>>>>>>     void init(RecordContext recordContext);
> >>>>>>>     void close();
> >>>>>>> }
> >>>>>>>
> >>>>>>> public interface RecordContext {
> >>>>>>>     String applicationId();
> >>>>>>>     TaskId taskId();
> >>>>>>>     StreamsMetrics metrics();
> >>>>>>>     String topic();
> >>>>>>>     int partition();
> >>>>>>>     long offset();
> >>>>>>>     long timestamp();
> >>>>>>>     Map<String, Object> appConfigs();
> >>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
> >>>>>>> }
> >>>>>>>
> >>>>>>>
> >>>>>>> public interface ProcessorContext extends RecordContext {
> >>>>>>>    // all methods but the ones in RecordContext
> >>>>>>> }
> >>>>>>>
> >>>>>>> As a result:
> >>>>>>> * . All "withKey" and "withoutKey" interfaces can be converted to
> >>>> their
> >>>>>>> Rich counterparts (with empty init() and close() methods)
> >>>>>>> *. All related Processors will accept Rich interfaces in their
> >>>>>>> constructors.
> >>>>>>> *. So, we convert the related "withKey" or "withoutKey" interfaces
> >>> to
> >>>>> Rich
> >>>>>>> interface while building the topology and initialize the related
> >>>>> processors
> >>>>>>> with Rich interfaces only.
> >>>>>>> *. We will not need to overloaded methods for rich functions as
> Rich
> >>>>>>> interfaces extend withKey interfaces. We will just check the object
> >>>> type
> >>>>>>> and act accordingly.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 3. There was some thoughts that the above approach does not support
> >>>>> lambdas
> >>>>>>> so we should support only one method, only init(RecordContext), as
> >>>> part
> >>>>> of
> >>>>>>> Rich interfaces.
> >>>>>>> This is still in discussion. Personally I think Rich interfaces are
> >>> by
> >>>>>>> definition lambda-free and we should not care much about it.
> >>>>>>>
> >>>>>>>
> >>>>>>> 4. Thanks to Matthias's discussion, an alternative we considered
> was
> >>>> to
> >>>>>>> pass in the RecordContext as method parameter.  This might even
> >>> allow
> >>>> to
> >>>>>>> use Lambdas and we could keep the name RichFunction as we preserve
> >>> the
> >>>>>>> nature of being a function.
> >>>>>>> "If you go with `init()` and `close()` we basically
> >>>>>>> allow users to have an in-memory state for a function. Thus, we
> >>> cannot
> >>>>>>> share a single instance of RichValueMapper (etc) over multiple
> tasks
> >>>> and
> >>>>>>> we would need a supplier pattern similar to #transform(). And this
> >>>> would
> >>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
> >>>>>>> inherit from ValueMapper and thus we would need many new overload
> >>> for
> >>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
> >>>>>>>
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Jeyhun
> >>>>>>>
> >>>>>>>
> >>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> >>> matthias@confluent.io
> >>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> 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 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:
> >>>>>>>>>
> >>>>>>>>> ---------------------------
> >>>>>>>>>
> >>>>>>>>> interface RichValueMapper<V, VR> {
> >>>>>>>>>
> >>>>>>>>> VR apply(final V value, final RecordContext context);
> >>>>>>>>>
> >>>>>>>>> }
> >>>>>>>>>
> >>>>>>>>> ...
> >>>>>>>>>
> >>>>>>>>> // then in KStreams
> >>>>>>>>>
> >>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends
> VR>
> >>>>>>>> mapper);
> >>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
> >>>> V, ?
> >>>>>>>>> extends VR> mapper);
> >>>>>>>>>
> >>>>>>>>> -------------------------------
> >>>>>>>>>
> >>>>>>>>> The caveat is that it will introduces more overloads; but I think
> >>>> the
> >>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
> >>>>>>>>> state-store-supplier overides, both of which can be reduced in
> the
> >>>>> near
> >>>>>>>>> future, and I felt this overloading is still worthwhile, as it
> has
> >>>> the
> >>>>>>>>> following benefits:
> >>>>>>>>>
> >>>>>>>>> 1) still allow lambda expressions.
> >>>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
> >>>> functions
> >>>>>>> to
> >>>>>>>>> rich functions)
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Maybe this approach has already been discussed and I may have
> >>>>>>> overlooked
> >>>>>>>> in
> >>>>>>>>> the email thread; anyways, lmk.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> >>>>>>> matthias@confluent.io>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> 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
> >>>>>>>>>> to get an overall picture to converge to an overall consisted
> >>> API.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more overload.
> It
> >>>>>>> might
> >>>>>>>>>> be sufficient though, to do a single xxxWithContext() overload
> >>> that
> >>>>>>> will
> >>>>>>>>>> provide key+value+context. Otherwise, if might get too messy
> >>> having
> >>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> >>>>>>>>>> ValueMapperWithKeyWithContext.
> >>>>>>>>>>
> >>>>>>>>>> On the other hand, we also have the "builder pattern" idea as an
> >>>> API
> >>>>>>>>>> change and this might mitigate the overload problem. Not for
> >>> simple
> >>>>>>>>>> function like map/flatMap etc but for joins and aggregations.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On the other hand, as I mentioned in an older email, I am
> >>>> personally
> >>>>>>>>>> fine to break the pure functional interface, and add
> >>>>>>>>>>
> >>>>>>>>>>   - interface WithRecordContext with method
> `open(RecordContext)`
> >>>> (or
> >>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
> >>>>>>>>>>
> >>>>>>>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
> >>>>>>>>>> WithRecordContext
> >>>>>>>>>>
> >>>>>>>>>> This would allow us to avoid any overload. Of course, we don't
> >>> get
> >>>> a
> >>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> I am personally a little bit undecided what the better option
> >>> might
> >>>>>>> be.
> >>>>>>>>>> Curious to hear what other think about this trade off.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> >>>>>>>>>>> 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.
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Jeyhun
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> >>> wangguoz@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> 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 <
> >>>>>>> je.karimov@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>> 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 like to get your comments.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> [1]
> >>>>>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >>>>>>>>>> ValueTransformer+ValueMapper+
> >>>>>>>>>>>>> and+ValueJoiner
> >>>>>>>>>>>>> [2]
> >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>> --
> >>>>>>>>>>>>> -Cheers
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Jeyhun
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> --
> >>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>> --
> >>>>>>> -Cheers
> >>>>>>>
> >>>>>>> Jeyhun
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>> --
> >>>> -Cheers
> >>>>
> >>>> Jeyhun
> >>>>
> >>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >
>
> --
-Cheers

Jeyhun

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 will
be provided by the consecutive call to aggregate anyways.


-Matthias

On 6/29/17 1:11 PM, Matthias J. Sax wrote:
> 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 implement both. This allows us to cast
> it to `RecordContext` and thus limit the visible scope.
> 
> 
> -Matthias
> 
> 
> 
> On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
>> 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 address this issue as well or
>> continue as it is?
>>
>> Cheers,
>> Jeyhun
>>
>> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> LGTM. Thanks!
>>>
>>>
>>> Guozhang
>>>
>>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <je...@gmail.com>
>>> 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 KAFKA-3907 [1] can also be integrated into the
>>>> KIP by adding related method inside RecordContext interface.
>>>>
>>>>
>>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>>
>>>>
>>>> Cheers,
>>>> Jeyhun
>>>>
>>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>>> 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 information only. If you want to do a
>>>>> stateful computation you should rather use #transform().
>>>>>
>>>>> Furthermore, as pointed out, we would need to switch to a
>>>>> supplier-pattern introducing many more overloads.
>>>>>
>>>>> For those reason, I advocate for a simple interface with a single
>>> method
>>>>> that passes in a RecordContext object.
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>>>>>> 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 <je.karimov@gmail.com
>>>>
>>>>> wrote:
>>>>>>
>>>>>>> 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, so I will not mention them.
>>>>>>> The related discussions are spread in KIP-149 and in this KIP
>>>> (KIP-159)
>>>>>>> discussion threads.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 1. The idea of rich functions came into the stage with KIP-149, in
>>>>>>> discussion thread. As a result we extended KIP-149 to support Rich
>>>>>>> functions as well.
>>>>>>>
>>>>>>> 2.  To as part of the Rich functions, we provided init
>>>>> (ProcessorContext)
>>>>>>> method. Afterwards, Dammian suggested that we should not provide
>>>>>>> ProcessorContext to users. As a result, we separated the two
>>> problems
>>>>> into
>>>>>>> two separate KIPs, as it seems they can be solved in parallel.
>>>>>>>
>>>>>>> - One approach we considered was :
>>>>>>>
>>>>>>> public interface ValueMapperWithKey<K, V, VR> {
>>>>>>>     VR apply(final K key, final V value);
>>>>>>> }
>>>>>>>
>>>>>>> public interface RichValueMapper<K, V, VR> extends RichFunction{
>>>>>>> }
>>>>>>>
>>>>>>> public interface RichFunction {
>>>>>>>     void init(RecordContext recordContext);
>>>>>>>     void close();
>>>>>>> }
>>>>>>>
>>>>>>> public interface RecordContext {
>>>>>>>     String applicationId();
>>>>>>>     TaskId taskId();
>>>>>>>     StreamsMetrics metrics();
>>>>>>>     String topic();
>>>>>>>     int partition();
>>>>>>>     long offset();
>>>>>>>     long timestamp();
>>>>>>>     Map<String, Object> appConfigs();
>>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
>>>>>>> }
>>>>>>>
>>>>>>>
>>>>>>> public interface ProcessorContext extends RecordContext {
>>>>>>>    // all methods but the ones in RecordContext
>>>>>>> }
>>>>>>>
>>>>>>> As a result:
>>>>>>> * . All "withKey" and "withoutKey" interfaces can be converted to
>>>> their
>>>>>>> Rich counterparts (with empty init() and close() methods)
>>>>>>> *. All related Processors will accept Rich interfaces in their
>>>>>>> constructors.
>>>>>>> *. So, we convert the related "withKey" or "withoutKey" interfaces
>>> to
>>>>> Rich
>>>>>>> interface while building the topology and initialize the related
>>>>> processors
>>>>>>> with Rich interfaces only.
>>>>>>> *. We will not need to overloaded methods for rich functions as Rich
>>>>>>> interfaces extend withKey interfaces. We will just check the object
>>>> type
>>>>>>> and act accordingly.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 3. There was some thoughts that the above approach does not support
>>>>> lambdas
>>>>>>> so we should support only one method, only init(RecordContext), as
>>>> part
>>>>> of
>>>>>>> Rich interfaces.
>>>>>>> This is still in discussion. Personally I think Rich interfaces are
>>> by
>>>>>>> definition lambda-free and we should not care much about it.
>>>>>>>
>>>>>>>
>>>>>>> 4. Thanks to Matthias's discussion, an alternative we considered was
>>>> to
>>>>>>> pass in the RecordContext as method parameter.  This might even
>>> allow
>>>> to
>>>>>>> use Lambdas and we could keep the name RichFunction as we preserve
>>> the
>>>>>>> nature of being a function.
>>>>>>> "If you go with `init()` and `close()` we basically
>>>>>>> allow users to have an in-memory state for a function. Thus, we
>>> cannot
>>>>>>> share a single instance of RichValueMapper (etc) over multiple tasks
>>>> and
>>>>>>> we would need a supplier pattern similar to #transform(). And this
>>>> would
>>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
>>>>>>> inherit from ValueMapper and thus we would need many new overload
>>> for
>>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
>>>>>>>
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Jeyhun
>>>>>>>
>>>>>>>
>>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
>>> matthias@confluent.io
>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 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 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:
>>>>>>>>>
>>>>>>>>> ---------------------------
>>>>>>>>>
>>>>>>>>> interface RichValueMapper<V, VR> {
>>>>>>>>>
>>>>>>>>> VR apply(final V value, final RecordContext context);
>>>>>>>>>
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> ...
>>>>>>>>>
>>>>>>>>> // then in KStreams
>>>>>>>>>
>>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
>>>>>>>> mapper);
>>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
>>>> V, ?
>>>>>>>>> extends VR> mapper);
>>>>>>>>>
>>>>>>>>> -------------------------------
>>>>>>>>>
>>>>>>>>> The caveat is that it will introduces more overloads; but I think
>>>> the
>>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
>>>>>>>>> state-store-supplier overides, both of which can be reduced in the
>>>>> near
>>>>>>>>> future, and I felt this overloading is still worthwhile, as it has
>>>> the
>>>>>>>>> following benefits:
>>>>>>>>>
>>>>>>>>> 1) still allow lambda expressions.
>>>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
>>>> functions
>>>>>>> to
>>>>>>>>> rich functions)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Maybe this approach has already been discussed and I may have
>>>>>>> overlooked
>>>>>>>> in
>>>>>>>>> the email thread; anyways, lmk.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>>>>>>> matthias@confluent.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 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
>>>>>>>>>> to get an overall picture to converge to an overall consisted
>>> API.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
>>>>>>> might
>>>>>>>>>> be sufficient though, to do a single xxxWithContext() overload
>>> that
>>>>>>> will
>>>>>>>>>> provide key+value+context. Otherwise, if might get too messy
>>> having
>>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>>>>>>>>>> ValueMapperWithKeyWithContext.
>>>>>>>>>>
>>>>>>>>>> On the other hand, we also have the "builder pattern" idea as an
>>>> API
>>>>>>>>>> change and this might mitigate the overload problem. Not for
>>> simple
>>>>>>>>>> function like map/flatMap etc but for joins and aggregations.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On the other hand, as I mentioned in an older email, I am
>>>> personally
>>>>>>>>>> fine to break the pure functional interface, and add
>>>>>>>>>>
>>>>>>>>>>   - interface WithRecordContext with method `open(RecordContext)`
>>>> (or
>>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
>>>>>>>>>>
>>>>>>>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
>>>>>>>>>> WithRecordContext
>>>>>>>>>>
>>>>>>>>>> This would allow us to avoid any overload. Of course, we don't
>>> get
>>>> a
>>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> I am personally a little bit undecided what the better option
>>> might
>>>>>>> be.
>>>>>>>>>> Curious to hear what other think about this trade off.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>>>>>>>>>> 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.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
>>> wangguoz@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> 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 <
>>>>>>> je.karimov@gmail.com
>>>>>>>>>
>>>>>>>>>>>> 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 like to get your comments.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>>>>>>>>>> ValueTransformer+ValueMapper+
>>>>>>>>>>>>> and+ValueJoiner
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>> --
>>>>>>>>>>>>> -Cheers
>>>>>>>>>>>>>
>>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>> -Cheers
>>>>>>>
>>>>>>> Jeyhun
>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>> -Cheers
>>>>
>>>> Jeyhun
>>>>
>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
> 


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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 implement both. This allows us to cast
it to `RecordContext` and thus limit the visible scope.


-Matthias



On 6/27/17 1:35 PM, Jeyhun Karimov wrote:
> 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 address this issue as well or
> continue as it is?
> 
> Cheers,
> Jeyhun
> 
> On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com> wrote:
> 
>> LGTM. Thanks!
>>
>>
>> Guozhang
>>
>> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <je...@gmail.com>
>> 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 KAFKA-3907 [1] can also be integrated into the
>>> KIP by adding related method inside RecordContext interface.
>>>
>>>
>>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> 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 information only. If you want to do a
>>>> stateful computation you should rather use #transform().
>>>>
>>>> Furthermore, as pointed out, we would need to switch to a
>>>> supplier-pattern introducing many more overloads.
>>>>
>>>> For those reason, I advocate for a simple interface with a single
>> method
>>>> that passes in a RecordContext object.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>>>>> 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 <je.karimov@gmail.com
>>>
>>>> wrote:
>>>>>
>>>>>> 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, so I will not mention them.
>>>>>> The related discussions are spread in KIP-149 and in this KIP
>>> (KIP-159)
>>>>>> discussion threads.
>>>>>>
>>>>>>
>>>>>>
>>>>>> 1. The idea of rich functions came into the stage with KIP-149, in
>>>>>> discussion thread. As a result we extended KIP-149 to support Rich
>>>>>> functions as well.
>>>>>>
>>>>>> 2.  To as part of the Rich functions, we provided init
>>>> (ProcessorContext)
>>>>>> method. Afterwards, Dammian suggested that we should not provide
>>>>>> ProcessorContext to users. As a result, we separated the two
>> problems
>>>> into
>>>>>> two separate KIPs, as it seems they can be solved in parallel.
>>>>>>
>>>>>> - One approach we considered was :
>>>>>>
>>>>>> public interface ValueMapperWithKey<K, V, VR> {
>>>>>>     VR apply(final K key, final V value);
>>>>>> }
>>>>>>
>>>>>> public interface RichValueMapper<K, V, VR> extends RichFunction{
>>>>>> }
>>>>>>
>>>>>> public interface RichFunction {
>>>>>>     void init(RecordContext recordContext);
>>>>>>     void close();
>>>>>> }
>>>>>>
>>>>>> public interface RecordContext {
>>>>>>     String applicationId();
>>>>>>     TaskId taskId();
>>>>>>     StreamsMetrics metrics();
>>>>>>     String topic();
>>>>>>     int partition();
>>>>>>     long offset();
>>>>>>     long timestamp();
>>>>>>     Map<String, Object> appConfigs();
>>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
>>>>>> }
>>>>>>
>>>>>>
>>>>>> public interface ProcessorContext extends RecordContext {
>>>>>>    // all methods but the ones in RecordContext
>>>>>> }
>>>>>>
>>>>>> As a result:
>>>>>> * . All "withKey" and "withoutKey" interfaces can be converted to
>>> their
>>>>>> Rich counterparts (with empty init() and close() methods)
>>>>>> *. All related Processors will accept Rich interfaces in their
>>>>>> constructors.
>>>>>> *. So, we convert the related "withKey" or "withoutKey" interfaces
>> to
>>>> Rich
>>>>>> interface while building the topology and initialize the related
>>>> processors
>>>>>> with Rich interfaces only.
>>>>>> *. We will not need to overloaded methods for rich functions as Rich
>>>>>> interfaces extend withKey interfaces. We will just check the object
>>> type
>>>>>> and act accordingly.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> 3. There was some thoughts that the above approach does not support
>>>> lambdas
>>>>>> so we should support only one method, only init(RecordContext), as
>>> part
>>>> of
>>>>>> Rich interfaces.
>>>>>> This is still in discussion. Personally I think Rich interfaces are
>> by
>>>>>> definition lambda-free and we should not care much about it.
>>>>>>
>>>>>>
>>>>>> 4. Thanks to Matthias's discussion, an alternative we considered was
>>> to
>>>>>> pass in the RecordContext as method parameter.  This might even
>> allow
>>> to
>>>>>> use Lambdas and we could keep the name RichFunction as we preserve
>> the
>>>>>> nature of being a function.
>>>>>> "If you go with `init()` and `close()` we basically
>>>>>> allow users to have an in-memory state for a function. Thus, we
>> cannot
>>>>>> share a single instance of RichValueMapper (etc) over multiple tasks
>>> and
>>>>>> we would need a supplier pattern similar to #transform(). And this
>>> would
>>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
>>>>>> inherit from ValueMapper and thus we would need many new overload
>> for
>>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
>>>>>>
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>>
>>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
>> matthias@confluent.io
>>>>
>>>>>> wrote:
>>>>>>
>>>>>>> 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 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:
>>>>>>>>
>>>>>>>> ---------------------------
>>>>>>>>
>>>>>>>> interface RichValueMapper<V, VR> {
>>>>>>>>
>>>>>>>> VR apply(final V value, final RecordContext context);
>>>>>>>>
>>>>>>>> }
>>>>>>>>
>>>>>>>> ...
>>>>>>>>
>>>>>>>> // then in KStreams
>>>>>>>>
>>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
>>>>>>> mapper);
>>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
>>> V, ?
>>>>>>>> extends VR> mapper);
>>>>>>>>
>>>>>>>> -------------------------------
>>>>>>>>
>>>>>>>> The caveat is that it will introduces more overloads; but I think
>>> the
>>>>>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
>>>>>>>> state-store-supplier overides, both of which can be reduced in the
>>>> near
>>>>>>>> future, and I felt this overloading is still worthwhile, as it has
>>> the
>>>>>>>> following benefits:
>>>>>>>>
>>>>>>>> 1) still allow lambda expressions.
>>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
>>> functions
>>>>>> to
>>>>>>>> rich functions)
>>>>>>>>
>>>>>>>>
>>>>>>>> Maybe this approach has already been discussed and I may have
>>>>>> overlooked
>>>>>>> in
>>>>>>>> the email thread; anyways, lmk.
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>>>>>> matthias@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> 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
>>>>>>>>> to get an overall picture to converge to an overall consisted
>> API.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
>>>>>> might
>>>>>>>>> be sufficient though, to do a single xxxWithContext() overload
>> that
>>>>>> will
>>>>>>>>> provide key+value+context. Otherwise, if might get too messy
>> having
>>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>>>>>>>>> ValueMapperWithKeyWithContext.
>>>>>>>>>
>>>>>>>>> On the other hand, we also have the "builder pattern" idea as an
>>> API
>>>>>>>>> change and this might mitigate the overload problem. Not for
>> simple
>>>>>>>>> function like map/flatMap etc but for joins and aggregations.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On the other hand, as I mentioned in an older email, I am
>>> personally
>>>>>>>>> fine to break the pure functional interface, and add
>>>>>>>>>
>>>>>>>>>   - interface WithRecordContext with method `open(RecordContext)`
>>> (or
>>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
>>>>>>>>>
>>>>>>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
>>>>>>>>> WithRecordContext
>>>>>>>>>
>>>>>>>>> This would allow us to avoid any overload. Of course, we don't
>> get
>>> a
>>>>>>>>> "pure function" interface and also sacrifices Lambdas.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I am personally a little bit undecided what the better option
>> might
>>>>>> be.
>>>>>>>>> Curious to hear what other think about this trade off.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>>>>>>>>> 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.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Jeyhun
>>>>>>>>>>
>>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
>> wangguoz@gmail.com>
>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> 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 <
>>>>>> je.karimov@gmail.com
>>>>>>>>
>>>>>>>>>>> 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 like to get your comments.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>>>>>>>>> ValueTransformer+ValueMapper+
>>>>>>>>>>>> and+ValueJoiner
>>>>>>>>>>>> [2]
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>> --
>>>>>>>>>>>> -Cheers
>>>>>>>>>>>>
>>>>>>>>>>>> Jeyhun
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>> -Cheers
>>>>>>
>>>>>> Jeyhun
>>>>>>
>>>>>
>>>>>
>>>>>
>>>>
>>>> --
>>> -Cheers
>>>
>>> Jeyhun
>>>
>>
>>
>>
>> --
>> -- Guozhang
>>


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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 address this issue as well or
continue as it is?

Cheers,
Jeyhun

On Wed, Jun 14, 2017 at 11:29 PM Guozhang Wang <wa...@gmail.com> wrote:

> LGTM. Thanks!
>
>
> Guozhang
>
> On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <je...@gmail.com>
> 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 KAFKA-3907 [1] can also be integrated into the
> > KIP by adding related method inside RecordContext interface.
> >
> >
> > [1] https://issues.apache.org/jira/browse/KAFKA-3907
> >
> >
> > Cheers,
> > Jeyhun
> >
> > On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > 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 information only. If you want to do a
> > > stateful computation you should rather use #transform().
> > >
> > > Furthermore, as pointed out, we would need to switch to a
> > > supplier-pattern introducing many more overloads.
> > >
> > > For those reason, I advocate for a simple interface with a single
> method
> > > that passes in a RecordContext object.
> > >
> > >
> > > -Matthias
> > >
> > >
> > > On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > > 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 <je.karimov@gmail.com
> >
> > > wrote:
> > > >
> > > >> 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, so I will not mention them.
> > > >> The related discussions are spread in KIP-149 and in this KIP
> > (KIP-159)
> > > >> discussion threads.
> > > >>
> > > >>
> > > >>
> > > >> 1. The idea of rich functions came into the stage with KIP-149, in
> > > >> discussion thread. As a result we extended KIP-149 to support Rich
> > > >> functions as well.
> > > >>
> > > >> 2.  To as part of the Rich functions, we provided init
> > > (ProcessorContext)
> > > >> method. Afterwards, Dammian suggested that we should not provide
> > > >> ProcessorContext to users. As a result, we separated the two
> problems
> > > into
> > > >> two separate KIPs, as it seems they can be solved in parallel.
> > > >>
> > > >> - One approach we considered was :
> > > >>
> > > >> public interface ValueMapperWithKey<K, V, VR> {
> > > >>     VR apply(final K key, final V value);
> > > >> }
> > > >>
> > > >> public interface RichValueMapper<K, V, VR> extends RichFunction{
> > > >> }
> > > >>
> > > >> public interface RichFunction {
> > > >>     void init(RecordContext recordContext);
> > > >>     void close();
> > > >> }
> > > >>
> > > >> public interface RecordContext {
> > > >>     String applicationId();
> > > >>     TaskId taskId();
> > > >>     StreamsMetrics metrics();
> > > >>     String topic();
> > > >>     int partition();
> > > >>     long offset();
> > > >>     long timestamp();
> > > >>     Map<String, Object> appConfigs();
> > > >>     Map<String, Object> appConfigsWithPrefix(String prefix);
> > > >> }
> > > >>
> > > >>
> > > >> public interface ProcessorContext extends RecordContext {
> > > >>    // all methods but the ones in RecordContext
> > > >> }
> > > >>
> > > >> As a result:
> > > >> * . All "withKey" and "withoutKey" interfaces can be converted to
> > their
> > > >> Rich counterparts (with empty init() and close() methods)
> > > >> *. All related Processors will accept Rich interfaces in their
> > > >> constructors.
> > > >> *. So, we convert the related "withKey" or "withoutKey" interfaces
> to
> > > Rich
> > > >> interface while building the topology and initialize the related
> > > processors
> > > >> with Rich interfaces only.
> > > >> *. We will not need to overloaded methods for rich functions as Rich
> > > >> interfaces extend withKey interfaces. We will just check the object
> > type
> > > >> and act accordingly.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> 3. There was some thoughts that the above approach does not support
> > > lambdas
> > > >> so we should support only one method, only init(RecordContext), as
> > part
> > > of
> > > >> Rich interfaces.
> > > >> This is still in discussion. Personally I think Rich interfaces are
> by
> > > >> definition lambda-free and we should not care much about it.
> > > >>
> > > >>
> > > >> 4. Thanks to Matthias's discussion, an alternative we considered was
> > to
> > > >> pass in the RecordContext as method parameter.  This might even
> allow
> > to
> > > >> use Lambdas and we could keep the name RichFunction as we preserve
> the
> > > >> nature of being a function.
> > > >> "If you go with `init()` and `close()` we basically
> > > >> allow users to have an in-memory state for a function. Thus, we
> cannot
> > > >> share a single instance of RichValueMapper (etc) over multiple tasks
> > and
> > > >> we would need a supplier pattern similar to #transform(). And this
> > would
> > > >> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
> > > >> inherit from ValueMapper and thus we would need many new overload
> for
> > > >> KStream/KTable classes". (Copy paste from Matthias's email)
> > > >>
> > > >>
> > > >> Cheers,
> > > >> Jeyhun
> > > >>
> > > >>
> > > >> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <
> matthias@confluent.io
> > >
> > > >> wrote:
> > > >>
> > > >>> 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 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:
> > > >>>>
> > > >>>> ---------------------------
> > > >>>>
> > > >>>> interface RichValueMapper<V, VR> {
> > > >>>>
> > > >>>> VR apply(final V value, final RecordContext context);
> > > >>>>
> > > >>>> }
> > > >>>>
> > > >>>> ...
> > > >>>>
> > > >>>> // then in KStreams
> > > >>>>
> > > >>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
> > > >>> mapper);
> > > >>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
> > V, ?
> > > >>>> extends VR> mapper);
> > > >>>>
> > > >>>> -------------------------------
> > > >>>>
> > > >>>> The caveat is that it will introduces more overloads; but I think
> > the
> > > >>>> #.overloads are mainly introduced by 1) serde overrides and 2)
> > > >>>> state-store-supplier overides, both of which can be reduced in the
> > > near
> > > >>>> future, and I felt this overloading is still worthwhile, as it has
> > the
> > > >>>> following benefits:
> > > >>>>
> > > >>>> 1) still allow lambda expressions.
> > > >>>> 2) clearer code path (do not need to "convert" from non-rich
> > functions
> > > >> to
> > > >>>> rich functions)
> > > >>>>
> > > >>>>
> > > >>>> Maybe this approach has already been discussed and I may have
> > > >> overlooked
> > > >>> in
> > > >>>> the email thread; anyways, lmk.
> > > >>>>
> > > >>>>
> > > >>>> Guozhang
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > > >> matthias@confluent.io>
> > > >>>> wrote:
> > > >>>>
> > > >>>>> 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
> > > >>>>> to get an overall picture to converge to an overall consisted
> API.
> > > >>>>>
> > > >>>>>
> > > >>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
> > > >> might
> > > >>>>> be sufficient though, to do a single xxxWithContext() overload
> that
> > > >> will
> > > >>>>> provide key+value+context. Otherwise, if might get too messy
> having
> > > >>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > > >>>>> ValueMapperWithKeyWithContext.
> > > >>>>>
> > > >>>>> On the other hand, we also have the "builder pattern" idea as an
> > API
> > > >>>>> change and this might mitigate the overload problem. Not for
> simple
> > > >>>>> function like map/flatMap etc but for joins and aggregations.
> > > >>>>>
> > > >>>>>
> > > >>>>> On the other hand, as I mentioned in an older email, I am
> > personally
> > > >>>>> fine to break the pure functional interface, and add
> > > >>>>>
> > > >>>>>   - interface WithRecordContext with method `open(RecordContext)`
> > (or
> > > >>>>> `init(...)`, or any better name) -- but not `close()`)
> > > >>>>>
> > > >>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
> > > >>>>> WithRecordContext
> > > >>>>>
> > > >>>>> This would allow us to avoid any overload. Of course, we don't
> get
> > a
> > > >>>>> "pure function" interface and also sacrifices Lambdas.
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> I am personally a little bit undecided what the better option
> might
> > > >> be.
> > > >>>>> Curious to hear what other think about this trade off.
> > > >>>>>
> > > >>>>>
> > > >>>>> -Matthias
> > > >>>>>
> > > >>>>>
> > > >>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > > >>>>>> 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.
> > > >>>>>>
> > > >>>>>> Cheers,
> > > >>>>>> Jeyhun
> > > >>>>>>
> > > >>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <
> wangguoz@gmail.com>
> > > >>> wrote:
> > > >>>>>>
> > > >>>>>>> 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 <
> > > >> je.karimov@gmail.com
> > > >>>>
> > > >>>>>>> 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 like to get your comments.
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> [1]
> > > >>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > > >>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > > >>>>> ValueTransformer+ValueMapper+
> > > >>>>>>>> and+ValueJoiner
> > > >>>>>>>> [2]
> > > >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> Cheers,
> > > >>>>>>>> Jeyhun
> > > >>>>>>>> --
> > > >>>>>>>> -Cheers
> > > >>>>>>>>
> > > >>>>>>>> Jeyhun
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> --
> > > >>>>>>> -- Guozhang
> > > >>>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>> --
> > > >> -Cheers
> > > >>
> > > >> Jeyhun
> > > >>
> > > >
> > > >
> > > >
> > >
> > > --
> > -Cheers
> >
> > Jeyhun
> >
>
>
>
> --
> -- Guozhang
>
-- 
-Cheers

Jeyhun

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 <je...@gmail.com>
> 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 KAFKA-3907 [1] can also be integrated into the
>> KIP by adding related method inside RecordContext interface.
>>
>>
>> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>>
>>
>> Cheers,
>> Jeyhun
>>
>> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>>> 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 information only. If you want to do a
>>> stateful computation you should rather use #transform().
>>>
>>> Furthermore, as pointed out, we would need to switch to a
>>> supplier-pattern introducing many more overloads.
>>>
>>> For those reason, I advocate for a simple interface with a single method
>>> that passes in a RecordContext object.
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 6/6/17 5:15 PM, Guozhang Wang wrote:
>>>> 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 <je...@gmail.com>
>>> wrote:
>>>>
>>>>> 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, so I will not mention them.
>>>>> The related discussions are spread in KIP-149 and in this KIP
>> (KIP-159)
>>>>> discussion threads.
>>>>>
>>>>>
>>>>>
>>>>> 1. The idea of rich functions came into the stage with KIP-149, in
>>>>> discussion thread. As a result we extended KIP-149 to support Rich
>>>>> functions as well.
>>>>>
>>>>> 2.  To as part of the Rich functions, we provided init
>>> (ProcessorContext)
>>>>> method. Afterwards, Dammian suggested that we should not provide
>>>>> ProcessorContext to users. As a result, we separated the two problems
>>> into
>>>>> two separate KIPs, as it seems they can be solved in parallel.
>>>>>
>>>>> - One approach we considered was :
>>>>>
>>>>> public interface ValueMapperWithKey<K, V, VR> {
>>>>>     VR apply(final K key, final V value);
>>>>> }
>>>>>
>>>>> public interface RichValueMapper<K, V, VR> extends RichFunction{
>>>>> }
>>>>>
>>>>> public interface RichFunction {
>>>>>     void init(RecordContext recordContext);
>>>>>     void close();
>>>>> }
>>>>>
>>>>> public interface RecordContext {
>>>>>     String applicationId();
>>>>>     TaskId taskId();
>>>>>     StreamsMetrics metrics();
>>>>>     String topic();
>>>>>     int partition();
>>>>>     long offset();
>>>>>     long timestamp();
>>>>>     Map<String, Object> appConfigs();
>>>>>     Map<String, Object> appConfigsWithPrefix(String prefix);
>>>>> }
>>>>>
>>>>>
>>>>> public interface ProcessorContext extends RecordContext {
>>>>>    // all methods but the ones in RecordContext
>>>>> }
>>>>>
>>>>> As a result:
>>>>> * . All "withKey" and "withoutKey" interfaces can be converted to
>> their
>>>>> Rich counterparts (with empty init() and close() methods)
>>>>> *. All related Processors will accept Rich interfaces in their
>>>>> constructors.
>>>>> *. So, we convert the related "withKey" or "withoutKey" interfaces to
>>> Rich
>>>>> interface while building the topology and initialize the related
>>> processors
>>>>> with Rich interfaces only.
>>>>> *. We will not need to overloaded methods for rich functions as Rich
>>>>> interfaces extend withKey interfaces. We will just check the object
>> type
>>>>> and act accordingly.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> 3. There was some thoughts that the above approach does not support
>>> lambdas
>>>>> so we should support only one method, only init(RecordContext), as
>> part
>>> of
>>>>> Rich interfaces.
>>>>> This is still in discussion. Personally I think Rich interfaces are by
>>>>> definition lambda-free and we should not care much about it.
>>>>>
>>>>>
>>>>> 4. Thanks to Matthias's discussion, an alternative we considered was
>> to
>>>>> pass in the RecordContext as method parameter.  This might even allow
>> to
>>>>> use Lambdas and we could keep the name RichFunction as we preserve the
>>>>> nature of being a function.
>>>>> "If you go with `init()` and `close()` we basically
>>>>> allow users to have an in-memory state for a function. Thus, we cannot
>>>>> share a single instance of RichValueMapper (etc) over multiple tasks
>> and
>>>>> we would need a supplier pattern similar to #transform(). And this
>> would
>>>>> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
>>>>> inherit from ValueMapper and thus we would need many new overload for
>>>>> KStream/KTable classes". (Copy paste from Matthias's email)
>>>>>
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>>
>>>>>
>>>>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <matthias@confluent.io
>>>
>>>>> wrote:
>>>>>
>>>>>> 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 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:
>>>>>>>
>>>>>>> ---------------------------
>>>>>>>
>>>>>>> interface RichValueMapper<V, VR> {
>>>>>>>
>>>>>>> VR apply(final V value, final RecordContext context);
>>>>>>>
>>>>>>> }
>>>>>>>
>>>>>>> ...
>>>>>>>
>>>>>>> // then in KStreams
>>>>>>>
>>>>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
>>>>>> mapper);
>>>>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
>> V, ?
>>>>>>> extends VR> mapper);
>>>>>>>
>>>>>>> -------------------------------
>>>>>>>
>>>>>>> The caveat is that it will introduces more overloads; but I think
>> the
>>>>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
>>>>>>> state-store-supplier overides, both of which can be reduced in the
>>> near
>>>>>>> future, and I felt this overloading is still worthwhile, as it has
>> the
>>>>>>> following benefits:
>>>>>>>
>>>>>>> 1) still allow lambda expressions.
>>>>>>> 2) clearer code path (do not need to "convert" from non-rich
>> functions
>>>>> to
>>>>>>> rich functions)
>>>>>>>
>>>>>>>
>>>>>>> Maybe this approach has already been discussed and I may have
>>>>> overlooked
>>>>>> in
>>>>>>> the email thread; anyways, lmk.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>>>>> matthias@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 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
>>>>>>>> to get an overall picture to converge to an overall consisted API.
>>>>>>>>
>>>>>>>>
>>>>>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
>>>>> might
>>>>>>>> be sufficient though, to do a single xxxWithContext() overload that
>>>>> will
>>>>>>>> provide key+value+context. Otherwise, if might get too messy having
>>>>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>>>>>>>> ValueMapperWithKeyWithContext.
>>>>>>>>
>>>>>>>> On the other hand, we also have the "builder pattern" idea as an
>> API
>>>>>>>> change and this might mitigate the overload problem. Not for simple
>>>>>>>> function like map/flatMap etc but for joins and aggregations.
>>>>>>>>
>>>>>>>>
>>>>>>>> On the other hand, as I mentioned in an older email, I am
>> personally
>>>>>>>> fine to break the pure functional interface, and add
>>>>>>>>
>>>>>>>>   - interface WithRecordContext with method `open(RecordContext)`
>> (or
>>>>>>>> `init(...)`, or any better name) -- but not `close()`)
>>>>>>>>
>>>>>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
>>>>>>>> WithRecordContext
>>>>>>>>
>>>>>>>> This would allow us to avoid any overload. Of course, we don't get
>> a
>>>>>>>> "pure function" interface and also sacrifices Lambdas.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> I am personally a little bit undecided what the better option might
>>>>> be.
>>>>>>>> Curious to hear what other think about this trade off.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>>>>>>>> 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.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Jeyhun
>>>>>>>>>
>>>>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 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 <
>>>>> je.karimov@gmail.com
>>>>>>>
>>>>>>>>>> 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 like to get your comments.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>>>>>>>> ValueTransformer+ValueMapper+
>>>>>>>>>>> and+ValueJoiner
>>>>>>>>>>> [2]
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Jeyhun
>>>>>>>>>>> --
>>>>>>>>>>> -Cheers
>>>>>>>>>>>
>>>>>>>>>>> Jeyhun
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>> --
>>>>> -Cheers
>>>>>
>>>>> Jeyhun
>>>>>
>>>>
>>>>
>>>>
>>>
>>> --
>> -Cheers
>>
>> Jeyhun
>>
> 
> 
> 


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

Posted by Guozhang Wang <wa...@gmail.com>.
LGTM. Thanks!


Guozhang

On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov <je...@gmail.com>
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 KAFKA-3907 [1] can also be integrated into the
> KIP by adding related method inside RecordContext interface.
>
>
> [1] https://issues.apache.org/jira/browse/KAFKA-3907
>
>
> Cheers,
> Jeyhun
>
> On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > 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 information only. If you want to do a
> > stateful computation you should rather use #transform().
> >
> > Furthermore, as pointed out, we would need to switch to a
> > supplier-pattern introducing many more overloads.
> >
> > For those reason, I advocate for a simple interface with a single method
> > that passes in a RecordContext object.
> >
> >
> > -Matthias
> >
> >
> > On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > > 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 <je...@gmail.com>
> > wrote:
> > >
> > >> 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, so I will not mention them.
> > >> The related discussions are spread in KIP-149 and in this KIP
> (KIP-159)
> > >> discussion threads.
> > >>
> > >>
> > >>
> > >> 1. The idea of rich functions came into the stage with KIP-149, in
> > >> discussion thread. As a result we extended KIP-149 to support Rich
> > >> functions as well.
> > >>
> > >> 2.  To as part of the Rich functions, we provided init
> > (ProcessorContext)
> > >> method. Afterwards, Dammian suggested that we should not provide
> > >> ProcessorContext to users. As a result, we separated the two problems
> > into
> > >> two separate KIPs, as it seems they can be solved in parallel.
> > >>
> > >> - One approach we considered was :
> > >>
> > >> public interface ValueMapperWithKey<K, V, VR> {
> > >>     VR apply(final K key, final V value);
> > >> }
> > >>
> > >> public interface RichValueMapper<K, V, VR> extends RichFunction{
> > >> }
> > >>
> > >> public interface RichFunction {
> > >>     void init(RecordContext recordContext);
> > >>     void close();
> > >> }
> > >>
> > >> public interface RecordContext {
> > >>     String applicationId();
> > >>     TaskId taskId();
> > >>     StreamsMetrics metrics();
> > >>     String topic();
> > >>     int partition();
> > >>     long offset();
> > >>     long timestamp();
> > >>     Map<String, Object> appConfigs();
> > >>     Map<String, Object> appConfigsWithPrefix(String prefix);
> > >> }
> > >>
> > >>
> > >> public interface ProcessorContext extends RecordContext {
> > >>    // all methods but the ones in RecordContext
> > >> }
> > >>
> > >> As a result:
> > >> * . All "withKey" and "withoutKey" interfaces can be converted to
> their
> > >> Rich counterparts (with empty init() and close() methods)
> > >> *. All related Processors will accept Rich interfaces in their
> > >> constructors.
> > >> *. So, we convert the related "withKey" or "withoutKey" interfaces to
> > Rich
> > >> interface while building the topology and initialize the related
> > processors
> > >> with Rich interfaces only.
> > >> *. We will not need to overloaded methods for rich functions as Rich
> > >> interfaces extend withKey interfaces. We will just check the object
> type
> > >> and act accordingly.
> > >>
> > >>
> > >>
> > >>
> > >> 3. There was some thoughts that the above approach does not support
> > lambdas
> > >> so we should support only one method, only init(RecordContext), as
> part
> > of
> > >> Rich interfaces.
> > >> This is still in discussion. Personally I think Rich interfaces are by
> > >> definition lambda-free and we should not care much about it.
> > >>
> > >>
> > >> 4. Thanks to Matthias's discussion, an alternative we considered was
> to
> > >> pass in the RecordContext as method parameter.  This might even allow
> to
> > >> use Lambdas and we could keep the name RichFunction as we preserve the
> > >> nature of being a function.
> > >> "If you go with `init()` and `close()` we basically
> > >> allow users to have an in-memory state for a function. Thus, we cannot
> > >> share a single instance of RichValueMapper (etc) over multiple tasks
> and
> > >> we would need a supplier pattern similar to #transform(). And this
> would
> > >> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
> > >> inherit from ValueMapper and thus we would need many new overload for
> > >> KStream/KTable classes". (Copy paste from Matthias's email)
> > >>
> > >>
> > >> Cheers,
> > >> Jeyhun
> > >>
> > >>
> > >> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <matthias@confluent.io
> >
> > >> wrote:
> > >>
> > >>> 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 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:
> > >>>>
> > >>>> ---------------------------
> > >>>>
> > >>>> interface RichValueMapper<V, VR> {
> > >>>>
> > >>>> VR apply(final V value, final RecordContext context);
> > >>>>
> > >>>> }
> > >>>>
> > >>>> ...
> > >>>>
> > >>>> // then in KStreams
> > >>>>
> > >>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
> > >>> mapper);
> > >>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super
> V, ?
> > >>>> extends VR> mapper);
> > >>>>
> > >>>> -------------------------------
> > >>>>
> > >>>> The caveat is that it will introduces more overloads; but I think
> the
> > >>>> #.overloads are mainly introduced by 1) serde overrides and 2)
> > >>>> state-store-supplier overides, both of which can be reduced in the
> > near
> > >>>> future, and I felt this overloading is still worthwhile, as it has
> the
> > >>>> following benefits:
> > >>>>
> > >>>> 1) still allow lambda expressions.
> > >>>> 2) clearer code path (do not need to "convert" from non-rich
> functions
> > >> to
> > >>>> rich functions)
> > >>>>
> > >>>>
> > >>>> Maybe this approach has already been discussed and I may have
> > >> overlooked
> > >>> in
> > >>>> the email thread; anyways, lmk.
> > >>>>
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>>
> > >>>>
> > >>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> > >> matthias@confluent.io>
> > >>>> wrote:
> > >>>>
> > >>>>> 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
> > >>>>> to get an overall picture to converge to an overall consisted API.
> > >>>>>
> > >>>>>
> > >>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
> > >> might
> > >>>>> be sufficient though, to do a single xxxWithContext() overload that
> > >> will
> > >>>>> provide key+value+context. Otherwise, if might get too messy having
> > >>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > >>>>> ValueMapperWithKeyWithContext.
> > >>>>>
> > >>>>> On the other hand, we also have the "builder pattern" idea as an
> API
> > >>>>> change and this might mitigate the overload problem. Not for simple
> > >>>>> function like map/flatMap etc but for joins and aggregations.
> > >>>>>
> > >>>>>
> > >>>>> On the other hand, as I mentioned in an older email, I am
> personally
> > >>>>> fine to break the pure functional interface, and add
> > >>>>>
> > >>>>>   - interface WithRecordContext with method `open(RecordContext)`
> (or
> > >>>>> `init(...)`, or any better name) -- but not `close()`)
> > >>>>>
> > >>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
> > >>>>> WithRecordContext
> > >>>>>
> > >>>>> This would allow us to avoid any overload. Of course, we don't get
> a
> > >>>>> "pure function" interface and also sacrifices Lambdas.
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> I am personally a little bit undecided what the better option might
> > >> be.
> > >>>>> Curious to hear what other think about this trade off.
> > >>>>>
> > >>>>>
> > >>>>> -Matthias
> > >>>>>
> > >>>>>
> > >>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > >>>>>> 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.
> > >>>>>>
> > >>>>>> Cheers,
> > >>>>>> Jeyhun
> > >>>>>>
> > >>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
> > >>> wrote:
> > >>>>>>
> > >>>>>>> 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 <
> > >> je.karimov@gmail.com
> > >>>>
> > >>>>>>> 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 like to get your comments.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> [1]
> > >>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > >>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > >>>>> ValueTransformer+ValueMapper+
> > >>>>>>>> and+ValueJoiner
> > >>>>>>>> [2]
> > >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Jeyhun
> > >>>>>>>> --
> > >>>>>>>> -Cheers
> > >>>>>>>>
> > >>>>>>>> Jeyhun
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> --
> > >>>>>>> -- Guozhang
> > >>>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>> --
> > >> -Cheers
> > >>
> > >> Jeyhun
> > >>
> > >
> > >
> > >
> >
> > --
> -Cheers
>
> Jeyhun
>



-- 
-- Guozhang

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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 RecordContext interface.


[1] https://issues.apache.org/jira/browse/KAFKA-3907


Cheers,
Jeyhun

On Tue, Jun 13, 2017 at 7:50 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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 information only. If you want to do a
> stateful computation you should rather use #transform().
>
> Furthermore, as pointed out, we would need to switch to a
> supplier-pattern introducing many more overloads.
>
> For those reason, I advocate for a simple interface with a single method
> that passes in a RecordContext object.
>
>
> -Matthias
>
>
> On 6/6/17 5:15 PM, Guozhang Wang wrote:
> > 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 <je...@gmail.com>
> wrote:
> >
> >> 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, so I will not mention them.
> >> The related discussions are spread in KIP-149 and in this KIP (KIP-159)
> >> discussion threads.
> >>
> >>
> >>
> >> 1. The idea of rich functions came into the stage with KIP-149, in
> >> discussion thread. As a result we extended KIP-149 to support Rich
> >> functions as well.
> >>
> >> 2.  To as part of the Rich functions, we provided init
> (ProcessorContext)
> >> method. Afterwards, Dammian suggested that we should not provide
> >> ProcessorContext to users. As a result, we separated the two problems
> into
> >> two separate KIPs, as it seems they can be solved in parallel.
> >>
> >> - One approach we considered was :
> >>
> >> public interface ValueMapperWithKey<K, V, VR> {
> >>     VR apply(final K key, final V value);
> >> }
> >>
> >> public interface RichValueMapper<K, V, VR> extends RichFunction{
> >> }
> >>
> >> public interface RichFunction {
> >>     void init(RecordContext recordContext);
> >>     void close();
> >> }
> >>
> >> public interface RecordContext {
> >>     String applicationId();
> >>     TaskId taskId();
> >>     StreamsMetrics metrics();
> >>     String topic();
> >>     int partition();
> >>     long offset();
> >>     long timestamp();
> >>     Map<String, Object> appConfigs();
> >>     Map<String, Object> appConfigsWithPrefix(String prefix);
> >> }
> >>
> >>
> >> public interface ProcessorContext extends RecordContext {
> >>    // all methods but the ones in RecordContext
> >> }
> >>
> >> As a result:
> >> * . All "withKey" and "withoutKey" interfaces can be converted to their
> >> Rich counterparts (with empty init() and close() methods)
> >> *. All related Processors will accept Rich interfaces in their
> >> constructors.
> >> *. So, we convert the related "withKey" or "withoutKey" interfaces to
> Rich
> >> interface while building the topology and initialize the related
> processors
> >> with Rich interfaces only.
> >> *. We will not need to overloaded methods for rich functions as Rich
> >> interfaces extend withKey interfaces. We will just check the object type
> >> and act accordingly.
> >>
> >>
> >>
> >>
> >> 3. There was some thoughts that the above approach does not support
> lambdas
> >> so we should support only one method, only init(RecordContext), as part
> of
> >> Rich interfaces.
> >> This is still in discussion. Personally I think Rich interfaces are by
> >> definition lambda-free and we should not care much about it.
> >>
> >>
> >> 4. Thanks to Matthias's discussion, an alternative we considered was to
> >> pass in the RecordContext as method parameter.  This might even allow to
> >> use Lambdas and we could keep the name RichFunction as we preserve the
> >> nature of being a function.
> >> "If you go with `init()` and `close()` we basically
> >> allow users to have an in-memory state for a function. Thus, we cannot
> >> share a single instance of RichValueMapper (etc) over multiple tasks and
> >> we would need a supplier pattern similar to #transform(). And this would
> >> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
> >> inherit from ValueMapper and thus we would need many new overload for
> >> KStream/KTable classes". (Copy paste from Matthias's email)
> >>
> >>
> >> Cheers,
> >> Jeyhun
> >>
> >>
> >> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <ma...@confluent.io>
> >> wrote:
> >>
> >>> 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 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:
> >>>>
> >>>> ---------------------------
> >>>>
> >>>> interface RichValueMapper<V, VR> {
> >>>>
> >>>> VR apply(final V value, final RecordContext context);
> >>>>
> >>>> }
> >>>>
> >>>> ...
> >>>>
> >>>> // then in KStreams
> >>>>
> >>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
> >>> mapper);
> >>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
> >>>> extends VR> mapper);
> >>>>
> >>>> -------------------------------
> >>>>
> >>>> The caveat is that it will introduces more overloads; but I think the
> >>>> #.overloads are mainly introduced by 1) serde overrides and 2)
> >>>> state-store-supplier overides, both of which can be reduced in the
> near
> >>>> future, and I felt this overloading is still worthwhile, as it has the
> >>>> following benefits:
> >>>>
> >>>> 1) still allow lambda expressions.
> >>>> 2) clearer code path (do not need to "convert" from non-rich functions
> >> to
> >>>> rich functions)
> >>>>
> >>>>
> >>>> Maybe this approach has already been discussed and I may have
> >> overlooked
> >>> in
> >>>> the email thread; anyways, lmk.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>>
> >>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> >> matthias@confluent.io>
> >>>> wrote:
> >>>>
> >>>>> 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
> >>>>> to get an overall picture to converge to an overall consisted API.
> >>>>>
> >>>>>
> >>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
> >> might
> >>>>> be sufficient though, to do a single xxxWithContext() overload that
> >> will
> >>>>> provide key+value+context. Otherwise, if might get too messy having
> >>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> >>>>> ValueMapperWithKeyWithContext.
> >>>>>
> >>>>> On the other hand, we also have the "builder pattern" idea as an API
> >>>>> change and this might mitigate the overload problem. Not for simple
> >>>>> function like map/flatMap etc but for joins and aggregations.
> >>>>>
> >>>>>
> >>>>> On the other hand, as I mentioned in an older email, I am personally
> >>>>> fine to break the pure functional interface, and add
> >>>>>
> >>>>>   - interface WithRecordContext with method `open(RecordContext)` (or
> >>>>> `init(...)`, or any better name) -- but not `close()`)
> >>>>>
> >>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
> >>>>> WithRecordContext
> >>>>>
> >>>>> This would allow us to avoid any overload. Of course, we don't get a
> >>>>> "pure function" interface and also sacrifices Lambdas.
> >>>>>
> >>>>>
> >>>>>
> >>>>> I am personally a little bit undecided what the better option might
> >> be.
> >>>>> Curious to hear what other think about this trade off.
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> >>>>>> 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.
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Jeyhun
> >>>>>>
> >>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
> >>> wrote:
> >>>>>>
> >>>>>>> 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 <
> >> je.karimov@gmail.com
> >>>>
> >>>>>>> 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 like to get your comments.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> [1]
> >>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >>>>> ValueTransformer+ValueMapper+
> >>>>>>>> and+ValueJoiner
> >>>>>>>> [2]
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Jeyhun
> >>>>>>>> --
> >>>>>>>> -Cheers
> >>>>>>>>
> >>>>>>>> Jeyhun
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> --
> >>>>>>> -- Guozhang
> >>>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>> --
> >> -Cheers
> >>
> >> Jeyhun
> >>
> >
> >
> >
>
> --
-Cheers

Jeyhun

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 information only. If you want to do a
stateful computation you should rather use #transform().

Furthermore, as pointed out, we would need to switch to a
supplier-pattern introducing many more overloads.

For those reason, I advocate for a simple interface with a single method
that passes in a RecordContext object.


-Matthias


On 6/6/17 5:15 PM, Guozhang Wang wrote:
> 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 <je...@gmail.com> wrote:
> 
>> 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, so I will not mention them.
>> The related discussions are spread in KIP-149 and in this KIP (KIP-159)
>> discussion threads.
>>
>>
>>
>> 1. The idea of rich functions came into the stage with KIP-149, in
>> discussion thread. As a result we extended KIP-149 to support Rich
>> functions as well.
>>
>> 2.  To as part of the Rich functions, we provided init (ProcessorContext)
>> method. Afterwards, Dammian suggested that we should not provide
>> ProcessorContext to users. As a result, we separated the two problems into
>> two separate KIPs, as it seems they can be solved in parallel.
>>
>> - One approach we considered was :
>>
>> public interface ValueMapperWithKey<K, V, VR> {
>>     VR apply(final K key, final V value);
>> }
>>
>> public interface RichValueMapper<K, V, VR> extends RichFunction{
>> }
>>
>> public interface RichFunction {
>>     void init(RecordContext recordContext);
>>     void close();
>> }
>>
>> public interface RecordContext {
>>     String applicationId();
>>     TaskId taskId();
>>     StreamsMetrics metrics();
>>     String topic();
>>     int partition();
>>     long offset();
>>     long timestamp();
>>     Map<String, Object> appConfigs();
>>     Map<String, Object> appConfigsWithPrefix(String prefix);
>> }
>>
>>
>> public interface ProcessorContext extends RecordContext {
>>    // all methods but the ones in RecordContext
>> }
>>
>> As a result:
>> * . All "withKey" and "withoutKey" interfaces can be converted to their
>> Rich counterparts (with empty init() and close() methods)
>> *. All related Processors will accept Rich interfaces in their
>> constructors.
>> *. So, we convert the related "withKey" or "withoutKey" interfaces to Rich
>> interface while building the topology and initialize the related processors
>> with Rich interfaces only.
>> *. We will not need to overloaded methods for rich functions as Rich
>> interfaces extend withKey interfaces. We will just check the object type
>> and act accordingly.
>>
>>
>>
>>
>> 3. There was some thoughts that the above approach does not support lambdas
>> so we should support only one method, only init(RecordContext), as part of
>> Rich interfaces.
>> This is still in discussion. Personally I think Rich interfaces are by
>> definition lambda-free and we should not care much about it.
>>
>>
>> 4. Thanks to Matthias's discussion, an alternative we considered was to
>> pass in the RecordContext as method parameter.  This might even allow to
>> use Lambdas and we could keep the name RichFunction as we preserve the
>> nature of being a function.
>> "If you go with `init()` and `close()` we basically
>> allow users to have an in-memory state for a function. Thus, we cannot
>> share a single instance of RichValueMapper (etc) over multiple tasks and
>> we would need a supplier pattern similar to #transform(). And this would
>> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
>> inherit from ValueMapper and thus we would need many new overload for
>> KStream/KTable classes". (Copy paste from Matthias's email)
>>
>>
>> Cheers,
>> Jeyhun
>>
>>
>> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>>> 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 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:
>>>>
>>>> ---------------------------
>>>>
>>>> interface RichValueMapper<V, VR> {
>>>>
>>>> VR apply(final V value, final RecordContext context);
>>>>
>>>> }
>>>>
>>>> ...
>>>>
>>>> // then in KStreams
>>>>
>>>> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
>>> mapper);
>>>> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
>>>> extends VR> mapper);
>>>>
>>>> -------------------------------
>>>>
>>>> The caveat is that it will introduces more overloads; but I think the
>>>> #.overloads are mainly introduced by 1) serde overrides and 2)
>>>> state-store-supplier overides, both of which can be reduced in the near
>>>> future, and I felt this overloading is still worthwhile, as it has the
>>>> following benefits:
>>>>
>>>> 1) still allow lambda expressions.
>>>> 2) clearer code path (do not need to "convert" from non-rich functions
>> to
>>>> rich functions)
>>>>
>>>>
>>>> Maybe this approach has already been discussed and I may have
>> overlooked
>>> in
>>>> the email thread; anyways, lmk.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
>> matthias@confluent.io>
>>>> wrote:
>>>>
>>>>> 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
>>>>> to get an overall picture to converge to an overall consisted API.
>>>>>
>>>>>
>>>>> @Jeyhun: about the overloads. Yes, we might get more overload. It
>> might
>>>>> be sufficient though, to do a single xxxWithContext() overload that
>> will
>>>>> provide key+value+context. Otherwise, if might get too messy having
>>>>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>>>>> ValueMapperWithKeyWithContext.
>>>>>
>>>>> On the other hand, we also have the "builder pattern" idea as an API
>>>>> change and this might mitigate the overload problem. Not for simple
>>>>> function like map/flatMap etc but for joins and aggregations.
>>>>>
>>>>>
>>>>> On the other hand, as I mentioned in an older email, I am personally
>>>>> fine to break the pure functional interface, and add
>>>>>
>>>>>   - interface WithRecordContext with method `open(RecordContext)` (or
>>>>> `init(...)`, or any better name) -- but not `close()`)
>>>>>
>>>>>   - interface ValueMapperWithRecordContext extends ValueMapper,
>>>>> WithRecordContext
>>>>>
>>>>> This would allow us to avoid any overload. Of course, we don't get a
>>>>> "pure function" interface and also sacrifices Lambdas.
>>>>>
>>>>>
>>>>>
>>>>> I am personally a little bit undecided what the better option might
>> be.
>>>>> Curious to hear what other think about this trade off.
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>>>>> 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.
>>>>>>
>>>>>> Cheers,
>>>>>> Jeyhun
>>>>>>
>>>>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>>>>
>>>>>>> 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 <
>> je.karimov@gmail.com
>>>>
>>>>>>> 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 like to get your comments.
>>>>>>>>
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>>>>> ValueTransformer+ValueMapper+
>>>>>>>> and+ValueJoiner
>>>>>>>> [2]
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>>>>
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Jeyhun
>>>>>>>> --
>>>>>>>> -Cheers
>>>>>>>>
>>>>>>>> Jeyhun
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> -- Guozhang
>>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>
>>> --
>> -Cheers
>>
>> Jeyhun
>>
> 
> 
> 


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

Posted by Guozhang Wang <wa...@gmail.com>.
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 <je...@gmail.com> wrote:

> 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, so I will not mention them.
> The related discussions are spread in KIP-149 and in this KIP (KIP-159)
> discussion threads.
>
>
>
> 1. The idea of rich functions came into the stage with KIP-149, in
> discussion thread. As a result we extended KIP-149 to support Rich
> functions as well.
>
> 2.  To as part of the Rich functions, we provided init (ProcessorContext)
> method. Afterwards, Dammian suggested that we should not provide
> ProcessorContext to users. As a result, we separated the two problems into
> two separate KIPs, as it seems they can be solved in parallel.
>
> - One approach we considered was :
>
> public interface ValueMapperWithKey<K, V, VR> {
>     VR apply(final K key, final V value);
> }
>
> public interface RichValueMapper<K, V, VR> extends RichFunction{
> }
>
> public interface RichFunction {
>     void init(RecordContext recordContext);
>     void close();
> }
>
> public interface RecordContext {
>     String applicationId();
>     TaskId taskId();
>     StreamsMetrics metrics();
>     String topic();
>     int partition();
>     long offset();
>     long timestamp();
>     Map<String, Object> appConfigs();
>     Map<String, Object> appConfigsWithPrefix(String prefix);
> }
>
>
> public interface ProcessorContext extends RecordContext {
>    // all methods but the ones in RecordContext
> }
>
> As a result:
> * . All "withKey" and "withoutKey" interfaces can be converted to their
> Rich counterparts (with empty init() and close() methods)
> *. All related Processors will accept Rich interfaces in their
> constructors.
> *. So, we convert the related "withKey" or "withoutKey" interfaces to Rich
> interface while building the topology and initialize the related processors
> with Rich interfaces only.
> *. We will not need to overloaded methods for rich functions as Rich
> interfaces extend withKey interfaces. We will just check the object type
> and act accordingly.
>
>
>
>
> 3. There was some thoughts that the above approach does not support lambdas
> so we should support only one method, only init(RecordContext), as part of
> Rich interfaces.
> This is still in discussion. Personally I think Rich interfaces are by
> definition lambda-free and we should not care much about it.
>
>
> 4. Thanks to Matthias's discussion, an alternative we considered was to
> pass in the RecordContext as method parameter.  This might even allow to
> use Lambdas and we could keep the name RichFunction as we preserve the
> nature of being a function.
> "If you go with `init()` and `close()` we basically
> allow users to have an in-memory state for a function. Thus, we cannot
> share a single instance of RichValueMapper (etc) over multiple tasks and
> we would need a supplier pattern similar to #transform(). And this would
> "break the flow" of the API, as (Rich)ValueMapperSupplier would not
> inherit from ValueMapper and thus we would need many new overload for
> KStream/KTable classes". (Copy paste from Matthias's email)
>
>
> Cheers,
> Jeyhun
>
>
> On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > 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 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:
> > >
> > > ---------------------------
> > >
> > > interface RichValueMapper<V, VR> {
> > >
> > > VR apply(final V value, final RecordContext context);
> > >
> > > }
> > >
> > > ...
> > >
> > > // then in KStreams
> > >
> > > <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
> > mapper);
> > > <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
> > > extends VR> mapper);
> > >
> > > -------------------------------
> > >
> > > The caveat is that it will introduces more overloads; but I think the
> > > #.overloads are mainly introduced by 1) serde overrides and 2)
> > > state-store-supplier overides, both of which can be reduced in the near
> > > future, and I felt this overloading is still worthwhile, as it has the
> > > following benefits:
> > >
> > > 1) still allow lambda expressions.
> > > 2) clearer code path (do not need to "convert" from non-rich functions
> to
> > > rich functions)
> > >
> > >
> > > Maybe this approach has already been discussed and I may have
> overlooked
> > in
> > > the email thread; anyways, lmk.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <
> matthias@confluent.io>
> > > wrote:
> > >
> > >> 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
> > >> to get an overall picture to converge to an overall consisted API.
> > >>
> > >>
> > >> @Jeyhun: about the overloads. Yes, we might get more overload. It
> might
> > >> be sufficient though, to do a single xxxWithContext() overload that
> will
> > >> provide key+value+context. Otherwise, if might get too messy having
> > >> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> > >> ValueMapperWithKeyWithContext.
> > >>
> > >> On the other hand, we also have the "builder pattern" idea as an API
> > >> change and this might mitigate the overload problem. Not for simple
> > >> function like map/flatMap etc but for joins and aggregations.
> > >>
> > >>
> > >> On the other hand, as I mentioned in an older email, I am personally
> > >> fine to break the pure functional interface, and add
> > >>
> > >>   - interface WithRecordContext with method `open(RecordContext)` (or
> > >> `init(...)`, or any better name) -- but not `close()`)
> > >>
> > >>   - interface ValueMapperWithRecordContext extends ValueMapper,
> > >> WithRecordContext
> > >>
> > >> This would allow us to avoid any overload. Of course, we don't get a
> > >> "pure function" interface and also sacrifices Lambdas.
> > >>
> > >>
> > >>
> > >> I am personally a little bit undecided what the better option might
> be.
> > >> Curious to hear what other think about this trade off.
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > >>> 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.
> > >>>
> > >>> Cheers,
> > >>> Jeyhun
> > >>>
> > >>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>>
> > >>>> 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 <
> je.karimov@gmail.com
> > >
> > >>>> 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 like to get your comments.
> > >>>>>
> > >>>>>
> > >>>>> [1]
> > >>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > >>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> > >> ValueTransformer+ValueMapper+
> > >>>>> and+ValueJoiner
> > >>>>> [2]
> > >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>> 159%3A+Introducing+Rich+functions+to+Streams
> > >>>>>
> > >>>>>
> > >>>>> Cheers,
> > >>>>> Jeyhun
> > >>>>> --
> > >>>>> -Cheers
> > >>>>>
> > >>>>> Jeyhun
> > >>>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>> --
> > >>>> -- Guozhang
> > >>>>
> > >>
> > >>
> > >
> > >
> >
> > --
> -Cheers
>
> Jeyhun
>



-- 
-- Guozhang

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

Posted by Jeyhun Karimov <je...@gmail.com>.
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, so I will not mention them.
The related discussions are spread in KIP-149 and in this KIP (KIP-159)
discussion threads.



1. The idea of rich functions came into the stage with KIP-149, in
discussion thread. As a result we extended KIP-149 to support Rich
functions as well.

2.  To as part of the Rich functions, we provided init (ProcessorContext)
method. Afterwards, Dammian suggested that we should not provide
ProcessorContext to users. As a result, we separated the two problems into
two separate KIPs, as it seems they can be solved in parallel.

- One approach we considered was :

public interface ValueMapperWithKey<K, V, VR> {
    VR apply(final K key, final V value);
}

public interface RichValueMapper<K, V, VR> extends RichFunction{
}

public interface RichFunction {
    void init(RecordContext recordContext);
    void close();
}

public interface RecordContext {
    String applicationId();
    TaskId taskId();
    StreamsMetrics metrics();
    String topic();
    int partition();
    long offset();
    long timestamp();
    Map<String, Object> appConfigs();
    Map<String, Object> appConfigsWithPrefix(String prefix);
}


public interface ProcessorContext extends RecordContext {
   // all methods but the ones in RecordContext
}

As a result:
* . All "withKey" and "withoutKey" interfaces can be converted to their
Rich counterparts (with empty init() and close() methods)
*. All related Processors will accept Rich interfaces in their constructors.
*. So, we convert the related "withKey" or "withoutKey" interfaces to Rich
interface while building the topology and initialize the related processors
with Rich interfaces only.
*. We will not need to overloaded methods for rich functions as Rich
interfaces extend withKey interfaces. We will just check the object type
and act accordingly.




3. There was some thoughts that the above approach does not support lambdas
so we should support only one method, only init(RecordContext), as part of
Rich interfaces.
This is still in discussion. Personally I think Rich interfaces are by
definition lambda-free and we should not care much about it.


4. Thanks to Matthias's discussion, an alternative we considered was to
pass in the RecordContext as method parameter.  This might even allow to
use Lambdas and we could keep the name RichFunction as we preserve the
nature of being a function.
"If you go with `init()` and `close()` we basically
allow users to have an in-memory state for a function. Thus, we cannot
share a single instance of RichValueMapper (etc) over multiple tasks and
we would need a supplier pattern similar to #transform(). And this would
"break the flow" of the API, as (Rich)ValueMapperSupplier would not
inherit from ValueMapper and thus we would need many new overload for
KStream/KTable classes". (Copy paste from Matthias's email)


Cheers,
Jeyhun


On Mon, Jun 5, 2017 at 5:18 AM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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 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:
> >
> > ---------------------------
> >
> > interface RichValueMapper<V, VR> {
> >
> > VR apply(final V value, final RecordContext context);
> >
> > }
> >
> > ...
> >
> > // then in KStreams
> >
> > <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR>
> mapper);
> > <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
> > extends VR> mapper);
> >
> > -------------------------------
> >
> > The caveat is that it will introduces more overloads; but I think the
> > #.overloads are mainly introduced by 1) serde overrides and 2)
> > state-store-supplier overides, both of which can be reduced in the near
> > future, and I felt this overloading is still worthwhile, as it has the
> > following benefits:
> >
> > 1) still allow lambda expressions.
> > 2) clearer code path (do not need to "convert" from non-rich functions to
> > rich functions)
> >
> >
> > Maybe this approach has already been discussed and I may have overlooked
> in
> > the email thread; anyways, lmk.
> >
> >
> > Guozhang
> >
> >
> >
> > On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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
> >> to get an overall picture to converge to an overall consisted API.
> >>
> >>
> >> @Jeyhun: about the overloads. Yes, we might get more overload. It might
> >> be sufficient though, to do a single xxxWithContext() overload that will
> >> provide key+value+context. Otherwise, if might get too messy having
> >> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> >> ValueMapperWithKeyWithContext.
> >>
> >> On the other hand, we also have the "builder pattern" idea as an API
> >> change and this might mitigate the overload problem. Not for simple
> >> function like map/flatMap etc but for joins and aggregations.
> >>
> >>
> >> On the other hand, as I mentioned in an older email, I am personally
> >> fine to break the pure functional interface, and add
> >>
> >>   - interface WithRecordContext with method `open(RecordContext)` (or
> >> `init(...)`, or any better name) -- but not `close()`)
> >>
> >>   - interface ValueMapperWithRecordContext extends ValueMapper,
> >> WithRecordContext
> >>
> >> This would allow us to avoid any overload. Of course, we don't get a
> >> "pure function" interface and also sacrifices Lambdas.
> >>
> >>
> >>
> >> I am personally a little bit undecided what the better option might be.
> >> Curious to hear what other think about this trade off.
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> >>> 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.
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>>
> >>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>>
> >>>> 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 <je.karimov@gmail.com
> >
> >>>> 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 like to get your comments.
> >>>>>
> >>>>>
> >>>>> [1]
> >>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> >> ValueTransformer+ValueMapper+
> >>>>> and+ValueJoiner
> >>>>> [2]
> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 159%3A+Introducing+Rich+functions+to+Streams
> >>>>>
> >>>>>
> >>>>> Cheers,
> >>>>> Jeyhun
> >>>>> --
> >>>>> -Cheers
> >>>>>
> >>>>> Jeyhun
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>
> >>
> >
> >
>
> --
-Cheers

Jeyhun

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 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:
> 
> ---------------------------
> 
> interface RichValueMapper<V, VR> {
> 
> VR apply(final V value, final RecordContext context);
> 
> }
> 
> ...
> 
> // then in KStreams
> 
> <VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR> mapper);
> <VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
> extends VR> mapper);
> 
> -------------------------------
> 
> The caveat is that it will introduces more overloads; but I think the
> #.overloads are mainly introduced by 1) serde overrides and 2)
> state-store-supplier overides, both of which can be reduced in the near
> future, and I felt this overloading is still worthwhile, as it has the
> following benefits:
> 
> 1) still allow lambda expressions.
> 2) clearer code path (do not need to "convert" from non-rich functions to
> rich functions)
> 
> 
> Maybe this approach has already been discussed and I may have overlooked in
> the email thread; anyways, lmk.
> 
> 
> Guozhang
> 
> 
> 
> On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> 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
>> to get an overall picture to converge to an overall consisted API.
>>
>>
>> @Jeyhun: about the overloads. Yes, we might get more overload. It might
>> be sufficient though, to do a single xxxWithContext() overload that will
>> provide key+value+context. Otherwise, if might get too messy having
>> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
>> ValueMapperWithKeyWithContext.
>>
>> On the other hand, we also have the "builder pattern" idea as an API
>> change and this might mitigate the overload problem. Not for simple
>> function like map/flatMap etc but for joins and aggregations.
>>
>>
>> On the other hand, as I mentioned in an older email, I am personally
>> fine to break the pure functional interface, and add
>>
>>   - interface WithRecordContext with method `open(RecordContext)` (or
>> `init(...)`, or any better name) -- but not `close()`)
>>
>>   - interface ValueMapperWithRecordContext extends ValueMapper,
>> WithRecordContext
>>
>> This would allow us to avoid any overload. Of course, we don't get a
>> "pure function" interface and also sacrifices Lambdas.
>>
>>
>>
>> I am personally a little bit undecided what the better option might be.
>> Curious to hear what other think about this trade off.
>>
>>
>> -Matthias
>>
>>
>> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
>>> 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.
>>>
>>> Cheers,
>>> Jeyhun
>>>
>>> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com> wrote:
>>>
>>>> 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 <je...@gmail.com>
>>>> 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 like to get your comments.
>>>>>
>>>>>
>>>>> [1]
>>>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
>> ValueTransformer+ValueMapper+
>>>>> and+ValueJoiner
>>>>> [2]
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>>>
>>>>>
>>>>> Cheers,
>>>>> Jeyhun
>>>>> --
>>>>> -Cheers
>>>>>
>>>>> Jeyhun
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> -- Guozhang
>>>>
>>
>>
> 
> 


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

Posted by Guozhang Wang <wa...@gmail.com>.
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:

---------------------------

interface RichValueMapper<V, VR> {

VR apply(final V value, final RecordContext context);

}

...

// then in KStreams

<VR> KStream<K, VR> mapValues(ValueMapper<? super V, ? extends VR> mapper);
<VR> KStream<K, VR> mapValueswithContext(RichValueMapper <? super V, ?
extends VR> mapper);

-------------------------------

The caveat is that it will introduces more overloads; but I think the
#.overloads are mainly introduced by 1) serde overrides and 2)
state-store-supplier overides, both of which can be reduced in the near
future, and I felt this overloading is still worthwhile, as it has the
following benefits:

1) still allow lambda expressions.
2) clearer code path (do not need to "convert" from non-rich functions to
rich functions)


Maybe this approach has already been discussed and I may have overlooked in
the email thread; anyways, lmk.


Guozhang



On Thu, Jun 1, 2017 at 10:18 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> 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
> to get an overall picture to converge to an overall consisted API.
>
>
> @Jeyhun: about the overloads. Yes, we might get more overload. It might
> be sufficient though, to do a single xxxWithContext() overload that will
> provide key+value+context. Otherwise, if might get too messy having
> ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
> ValueMapperWithKeyWithContext.
>
> On the other hand, we also have the "builder pattern" idea as an API
> change and this might mitigate the overload problem. Not for simple
> function like map/flatMap etc but for joins and aggregations.
>
>
> On the other hand, as I mentioned in an older email, I am personally
> fine to break the pure functional interface, and add
>
>   - interface WithRecordContext with method `open(RecordContext)` (or
> `init(...)`, or any better name) -- but not `close()`)
>
>   - interface ValueMapperWithRecordContext extends ValueMapper,
> WithRecordContext
>
> This would allow us to avoid any overload. Of course, we don't get a
> "pure function" interface and also sacrifices Lambdas.
>
>
>
> I am personally a little bit undecided what the better option might be.
> Curious to hear what other think about this trade off.
>
>
> -Matthias
>
>
> On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> > 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.
> >
> > Cheers,
> > Jeyhun
> >
> > On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com> wrote:
> >
> >> 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 <je...@gmail.com>
> >> 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 like to get your comments.
> >>>
> >>>
> >>> [1]
> >>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> >>> Re+DISCUSS+KIP+149+Enabling+key+access+in+
> ValueTransformer+ValueMapper+
> >>> and+ValueJoiner
> >>> [2]
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 159%3A+Introducing+Rich+functions+to+Streams
> >>>
> >>>
> >>> Cheers,
> >>> Jeyhun
> >>> --
> >>> -Cheers
> >>>
> >>> Jeyhun
> >>>
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
>
>


-- 
-- Guozhang

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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
to get an overall picture to converge to an overall consisted API.


@Jeyhun: about the overloads. Yes, we might get more overload. It might
be sufficient though, to do a single xxxWithContext() overload that will
provide key+value+context. Otherwise, if might get too messy having
ValueMapper, ValueMapperWithKey, ValueMapperWithContext,
ValueMapperWithKeyWithContext.

On the other hand, we also have the "builder pattern" idea as an API
change and this might mitigate the overload problem. Not for simple
function like map/flatMap etc but for joins and aggregations.


On the other hand, as I mentioned in an older email, I am personally
fine to break the pure functional interface, and add

  - interface WithRecordContext with method `open(RecordContext)` (or
`init(...)`, or any better name) -- but not `close()`)

  - interface ValueMapperWithRecordContext extends ValueMapper,
WithRecordContext

This would allow us to avoid any overload. Of course, we don't get a
"pure function" interface and also sacrifices Lambdas.



I am personally a little bit undecided what the better option might be.
Curious to hear what other think about this trade off.


-Matthias


On 6/1/17 6:13 PM, Jeyhun Karimov wrote:
> 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.
> 
> Cheers,
> Jeyhun
> 
> On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com> wrote:
> 
>> 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 <je...@gmail.com>
>> 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 like to get your comments.
>>>
>>>
>>> [1]
>>> http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
>>> Re+DISCUSS+KIP+149+Enabling+key+access+in+ValueTransformer+ValueMapper+
>>> and+ValueJoiner
>>> [2]
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 159%3A+Introducing+Rich+functions+to+Streams
>>>
>>>
>>> Cheers,
>>> Jeyhun
>>> --
>>> -Cheers
>>>
>>> Jeyhun
>>>
>>
>>
>>
>> --
>> -- Guozhang
>>


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

Posted by Jeyhun Karimov <je...@gmail.com>.
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.

Cheers,
Jeyhun

On Fri, Jun 2, 2017 at 2:27 AM Guozhang Wang <wa...@gmail.com> wrote:

> 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 <je...@gmail.com>
> 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 like to get your comments.
> >
> >
> > [1]
> > http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=
> > Re+DISCUSS+KIP+149+Enabling+key+access+in+ValueTransformer+ValueMapper+
> > and+ValueJoiner
> > [2]
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 159%3A+Introducing+Rich+functions+to+Streams
> >
> >
> > Cheers,
> > Jeyhun
> > --
> > -Cheers
> >
> > Jeyhun
> >
>
>
>
> --
> -- Guozhang
>
-- 
-Cheers

Jeyhun