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 2018/09/01 18:44:11 UTC

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Yes Adam, that makes sense.

I think it may be better to have a working PR to review before we complete
the VOTE thread. In my previous experience a large feature like this are
mostly definitely going to miss some devils in the details in the design
and wiki discussion phases.

That would unfortunately mean that your implementations may need to be
modified / updated along with the review and further KIP discussion. I can
understand this can be painful, but that may be the best option we can do
to avoid as much work to be wasted as possible.


Guozhang


On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Guozhang
>
> By workflow I mean just the overall process of how the KIP is implemented.
> Any ideas on the ways to reduce the topic count, materializations, if there
> is a better way to resolve out-of-order than a highwater mark table, if the
> design philosophy of “keep everything encapsulated within the join
> function” is appropriate, etc. I can implement the changes that John
> suggested, but if my overall workflow is not acceptable I would rather
> address that before making minor changes.
>
> If this requires a full candidate PR ready to go to prod then I can make
> those changes. Hope that clears things up.
>
> Thanks
>
> Adam
>
> > On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com> wrote:
> >
> > Hi Adam,
> >
> > What do you mean by "additional comments on the workflow.", do you mean
> to
> > let other review your PR https://github.com/apache/kafka/pull/5527 ? Is
> is
> > ready for reviews?
> >
> >
> > Guozhang
> >
> > On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
> adam.bellemare@gmail.com>
> > wrote:
> >
> >> Okay, I will implement John's suggestion of namespacing the external
> >> headers prior to processing, and then removing the namespacing prior to
> >> emitting. A potential future KIP could be to provide this namespacing
> >> automatically.
> >>
> >> I would also appreciate any other additional comments on the workflow.
> My
> >> goal is suss out agreement prior to moving to a vote.
> >>
> >>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >>>
> >>> I like John's idea as well: for this KIP specifically as we do not
> expect
> >>> any other consumers to read the repartition topics externally, we can
> >>> slightly prefix the header to be safe, while keeping the additional
> cost
> >>> (note the header field is per-record, so any additional byte is
> >> per-record
> >>> as well) low.
> >>>
> >>>
> >>> Guozhang
> >>>
> >>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
> >> adam.bellemare@gmail.com
> >>>>
> >>> wrote:
> >>>
> >>>> Hi John
> >>>>
> >>>> That is an excellent idea. The header usage I propose would be limited
> >>>> entirely to internal topics, and this could very well be the solution
> >> to
> >>>> potential conflicts. If we do not officially reserve a prefix "__"
> >> then I
> >>>> think this would be the safest idea, as it would entirely avoid any
> >>>> accidents (perhaps if a company is using its own "__" prefix for other
> >>>> reasons).
> >>>>
> >>>> Thanks
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
> >> wrote:
> >>>>
> >>>>> Just a quick thought regarding headers:
> >>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
> >> can
> >>>>> still
> >>>>>> consider using some name patterns to reduce the likelihood as much
> >> as
> >>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
> >>>>>> "__internal_[name]"?
> >>>>>
> >>>>> I think there is a safe way to avoid conflicts, since these headers
> >> are
> >>>>> only needed in internal topics (I think):
> >>>>> For internal and changelog topics, we can namespace all headers:
> >>>>> * user-defined headers are namespaced as "external." + headerKey
> >>>>> * internal headers are namespaced as "internal." + headerKey
> >>>>>
> >>>>> This is a lot of characters, so we could use a sigil instead (e.g.,
> >> "_"
> >>>> for
> >>>>> internal, "~" for external)
> >>>>>
> >>>>> We simply apply the namespacing when we read user headers from
> >> external
> >>>>> topics into the topology and then de-namespace them before we emit
> >> them
> >>>> to
> >>>>> an external topic (via "to" or "through").
> >>>>> Now, it is not possible to collide with user-defined headers.
> >>>>>
> >>>>> That said, I'd also be fine with just reserving "__" as a header
> >> prefix
> >>>> and
> >>>>> not worrying about collisions.
> >>>>>
> >>>>> Thanks for the KIP,
> >>>>> -John
> >>>>>
> >>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
> >> Jan.Filipiak@trivago.com
> >>>>
> >>>>> wrote:
> >>>>>
> >>>>>> Still havent completly grabbed it.
> >>>>>> sorry will read more
> >>>>>>
> >>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
> >>>>>>> Cool stuff.
> >>>>>>>
> >>>>>>> I made some random remarks. Did not touch the core of the
> >> algorithm
> >>>>> yet.
> >>>>>>>
> >>>>>>> Will do Monday 100%
> >>>>>>>
> >>>>>>> I don't see Interactive Queries :) like that!
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
> >>>>>>>> I have submitted a PR with my code against trunk:
> >>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>
> >>>>>>>> Do I continue on this thread or do we begin a new one for
> >>>> discussion?
> >>>>>>>>
> >>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
> >>>>> Jan.Filipiak@trivago.com
> >>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> even before message headers, the option for me always existed
> >> to
> >>>>>>>>> just wrap
> >>>>>>>>> the messages into my own custom envelop.
> >>>>>>>>> So I of course thought this through. One sentence in your last
> >>>> email
> >>>>>>>>> triggered all the thought process I put in the back then
> >>>>>>>>> again to design it in the, what i think is the "kafka-way". It
> >>>> ended
> >>>>> up
> >>>>>>>>> ranting a little about what happened in the past.
> >>>>>>>>>
> >>>>>>>>> I see plenty of colleagues of mine falling into traps in the
> >> API,
> >>>>>>>>> that I
> >>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
> >>>>>>>>> feeling again. So I hope it gives you some insights into my
> >>> though
> >>>>>>>>> process. I am aware that since i never ported 213 to higher
> >>>>>>>>> streams version, I don't really have a steak here and
> >> initially I
> >>>>>>>>> didn't
> >>>>>>>>> feel like actually sending it. But maybe you can pull
> >>>>>>>>> something good from it.
> >>>>>>>>>
> >>>>>>>>>  Best jan
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
> >>>>>>>>>>
> >>>>>>>>>> @Jan
> >>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow includes
> >>>>>>>>>> information
> >>>>>>>>>> about which record is processed first? I understand about a
> >>>>>>>>>> CombinedKey
> >>>>>>>>>> with both the Foreign and Primary key, but I don't see how you
> >>>> track
> >>>>>>>>>> ordering metadata in there unless you actually included a
> >>> metadata
> >>>>>>>>>> field
> >>>>>>>>>> in
> >>>>>>>>>> the key type as well.
> >>>>>>>>>>
> >>>>>>>>>> @Guozhang
> >>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
> >> be
> >>>>>>>>>> used in
> >>>>>>>>>> just the user-space? It seems that it is possible that a
> >>> collision
> >>>>>>>>>> on the
> >>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
> >> instance,
> >>>> if
> >>>>> I
> >>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
> >>>> user
> >>>>>>>>>> already
> >>>>>>>>>> specified their own header with the same key name, then it
> >>> appears
> >>>>>>>>>> there
> >>>>>>>>>> would be a collision. (This is one of the issues I brought up
> >> in
> >>>>>>>>>> the KIP).
> >>>>>>>>>>
> >>>>>>>>>> --------------------------------
> >>>>>>>>>>
> >>>>>>>>>> I will be posting a prototype PR against trunk within the next
> >>> day
> >>>>>>>>>> or two.
> >>>>>>>>>> One thing I need to point out is that my design very strictly
> >>>> wraps
> >>>>>>>>>> the
> >>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
> >> function.
> >>>>>>>>>> There is
> >>>>>>>>>> no
> >>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
> >>> with
> >>>>>>>>>> regards
> >>>>>>>>>> to out-of-order processing and no need for the DSL user to
> >> even
> >>>> know
> >>>>>>>>>> what's
> >>>>>>>>>> going on inside of the function. The code simply returns the
> >>>>>>>>>> results of
> >>>>>>>>>> the
> >>>>>>>>>> join, keyed by the original key. Currently my API mirrors
> >>>>>>>>>> identically the
> >>>>>>>>>> format of the data returned by the regular join function, and
> >> I
> >>>>>>>>>> believe
> >>>>>>>>>> that this is very useful to many users of the DSL. It is my
> >>>>>>>>>> understanding
> >>>>>>>>>> that one of the main design goals of the DSL is to provide
> >>> higher
> >>>>>>>>>> level
> >>>>>>>>>> functionality without requiring the users to know exactly
> >> what's
> >>>>>>>>>> going on
> >>>>>>>>>> under the hood. With this in mind, I thought it best to solve
> >>>>>>>>>> ordering and
> >>>>>>>>>> partitioning problems within the function and eliminate the
> >>>>>>>>>> requirement
> >>>>>>>>>> for
> >>>>>>>>>> users to do additional work after the fact to resolve the
> >>> results
> >>>>>>>>>> of their
> >>>>>>>>>> join. Basically, I am assuming that most users of the DSL just
> >>>>>>>>>> "want it to
> >>>>>>>>>> work" and want it to be easy. I did this operating under the
> >>>>>>>>>> assumption
> >>>>>>>>>> that if a user truly wants to optimize their own workflow down
> >>> to
> >>>>> the
> >>>>>>>>>> finest details then they will break from strictly using the
> >> DSL
> >>>> and
> >>>>>>>>>> move
> >>>>>>>>>> down to the processors API.
> >>>>>>>>>>
> >>>>>>>>> I think. The abstraction is not powerful enough
> >>>>>>>>> to not have kafka specifics leak up The leak I currently think
> >>> this
> >>>>>>>>> has is
> >>>>>>>>> that you can not reliable prevent the delete coming out first,
> >>>>>>>>> before you emit the correct new record. As it is an abstraction
> >>>>>>>>> entirely
> >>>>>>>>> around kafka.
> >>>>>>>>> I can only recommend to not to. Honesty and simplicity should
> >>>> always
> >>>>> be
> >>>>>>>>> first prio
> >>>>>>>>> trying to hide this just makes it more complex, less
> >>> understandable
> >>>>> and
> >>>>>>>>> will lead to mistakes
> >>>>>>>>> in usage.
> >>>>>>>>>
> >>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
> >>>>>>>>> optimization stages.
> >>>>>>>>> Can someone give me an example of an optimisation that really
> >>> can't
> >>>>> be
> >>>>>>>>> handled by the user
> >>>>>>>>> constructing his topology differently?
> >>>>>>>>> Having reusable Processor API components accessible by the DSL
> >>> and
> >>>>>>>>> composable as
> >>>>>>>>> one likes is exactly where DSL should max out and KSQL should
> >> do
> >>>> the
> >>>>>>>>> next
> >>>>>>>>> step.
> >>>>>>>>> I find it very unprofessional from a software engineering
> >>> approach
> >>>>>>>>> to run
> >>>>>>>>> software where
> >>>>>>>>> you can not at least senseful reason about the inner workings
> >> of
> >>>> the
> >>>>>>>>> libraries used.
> >>>>>>>>> Gives this people have to read and understand in anyway, why
> >> try
> >>> to
> >>>>>>>>> hide
> >>>>>>>>> it?
> >>>>>>>>>
> >>>>>>>>> It really miss the beauty of 0.10 version DSL.
> >>>>>>>>> Apparently not a thing I can influence but just warn about.
> >>>>>>>>>
> >>>>>>>>> @gouzhang
> >>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
> >>> prune
> >>>>> from
> >>>>>>>>> stream app's
> >>>>>>>>> because people just keep passing Materialized's into all the
> >>>>>>>>> operations.
> >>>>>>>>> :D :'-(
> >>>>>>>>> I regret that I couldn't convince you guys back then. Plus this
> >>>> whole
> >>>>>>>>> entire topology as a floating
> >>>>>>>>> interface chain, never seen it anywhere :-/ :'(
> >>>>>>>>>
> >>>>>>>>> I don't know. I guess this is just me regretting to only have
> >>>>> 24h/day.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> I updated the KIP today with some points worth talking about,
> >>>> should
> >>>>>>>>> anyone
> >>>>>>>>>> be so inclined to check it out. Currently we are running this
> >>> code
> >>>>> in
> >>>>>>>>>> production to handle relational joins from our Kafka Connect
> >>>>>>>>>> topics, as
> >>>>>>>>>> per
> >>>>>>>>>> the original motivation of the KIP.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
> >>> wangguoz@gmail.com
> >>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hello Adam,
> >>>>>>>>>>> As for your question regarding GraphNodes, it is for
> >> extending
> >>>>>>>>>>> Streams
> >>>>>>>>>>> optimization framework. You can find more details on
> >>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
> >>>>>>>>>>>
> >>>>>>>>>>> The main idea is that instead of directly building up the
> >>>> "physical
> >>>>>>>>>>> topology" (represented as Topology in the public package, and
> >>>>>>>>>>> internally
> >>>>>>>>>>> built as the ProcessorTopology class) while users are
> >>> specifying
> >>>>> the
> >>>>>>>>>>> transformation operators, we first keep it as a "logical
> >>>> topology"
> >>>>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
> >>>> then
> >>>>>>>>>>> only
> >>>>>>>>>>> execute the optimization and the construction of the
> >> "physical"
> >>>>>>>>>>> Topology
> >>>>>>>>>>> when StreamsBuilder.build() is called.
> >>>>>>>>>>>
> >>>>>>>>>>> Back to your question, I think it makes more sense to add a
> >> new
> >>>>>>>>>>> type of
> >>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
> >>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology we
> >>>> will
> >>>>>>>>>>> have
> >>>>>>>>>>> multiple connected ProcessorNodes to represent a
> >> (foreign-key)
> >>>>>>>>>>> join, we
> >>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just a
> >>>>>>>>>>> couple of
> >>>>>>>>>>> them in the logical representation so that in the future we
> >> can
> >>>>>>>>>>> construct
> >>>>>>>>>>> the physical topology differently (e.g. having another way
> >> than
> >>>> the
> >>>>>>>>>>> current
> >>>>>>>>>>> distributed hash-join).
> >>>>>>>>>>>
> >>>>>>>>>>> -------------------------------------------------------
> >>>>>>>>>>>
> >>>>>>>>>>> Back to your questions to KIP-213, I think Jan has summarized
> >>> it
> >>>>>>>>>>> pretty-well. Note that back then we do not have headers
> >> support
> >>>> so
> >>>>> we
> >>>>>>>>>>> have
> >>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Guozhang
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
> >>>>>>>>>>> Filipiak<Ja...@trivago.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hi Adam,
> >>>>>>>>>>>> I love how you are on to this already! I resolve this by
> >>>>>>>>>>>> "key-widening"
> >>>>>>>>>>>> I
> >>>>>>>>>>>> treat the result of FKA,and FKB differently.
> >>>>>>>>>>>> As you can see the output of my join has a Combined Key and
> >>>>>>>>>>>> therefore I
> >>>>>>>>>>>> can resolve the "race condition" in a group by
> >>>>>>>>>>>> if I so desire.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I think this reflects more what happens under the hood and
> >>> makes
> >>>>>>>>>>>> it more
> >>>>>>>>>>>> clear to the user what is going on. The Idea
> >>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
> >>> from
> >>>>>>>>>>>> my POV
> >>>>>>>>>>>> unideal.
> >>>>>>>>>>>>
> >>>>>>>>>>>> To write into your example:
> >>>>>>>>>>>>
> >>>>>>>>>>>> key + A, null)
> >>>>>>>>>>>> (key +B, <joined On FK =B>)
> >>>>>>>>>>>>
> >>>>>>>>>>>> is what my output would look like.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hope that makes sense :D
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best Jan
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>> If you do not use headers or other metadata, how do you
> >>> ensure
> >>>>> that
> >>>>>>>>>>>>> changes
> >>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
> >>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
> >> you
> >>>>>>>>>>>>> need to
> >>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
> >>> (FK=B).
> >>>>>>>>>>>>> In my
> >>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
> >>> for
> >>>>> the
> >>>>>>>>>>>>> final
> >>>>>>>>>>>>> output to be in either order - the correctly updated joined
> >>>>>>>>>>>>> value, or
> >>>>>>>>>>>>>
> >>>>>>>>>>>> the
> >>>>>>>>>>>> null for the delete.
> >>>>>>>>>>>>> (key, null)
> >>>>>>>>>>>>> (key, <joined On FK =B>)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> or
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> (key, <joined On FK =B>)
> >>>>>>>>>>>>> (key, null)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I looked back through your code and through the discussion
> >>>>>>>>>>>>> threads, and
> >>>>>>>>>>>>> didn't see any information on how you resolved this. I
> >> have a
> >>>>>>>>>>>>> version
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>> my
> >>>>>>>>>>>>> code working for 2.0, I am just adding more integration
> >> tests
> >>>>>>>>>>>>> and will
> >>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
> >> on
> >>>>>>>>>>>>> resolving
> >>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
> >>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Happy to see that you want to make an effort here.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
> >> not
> >>>>>>>>>>>>>> rewrite
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>> joiners + the merger.
> >>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
> >> if
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> repartition
> >>>>>>>>>>>> is optimized in 2.0 now.
> >>>>>>>>>>>>>> I made this
> >>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> 241+
> >>>>>>>>>>>>>> KTable+repartition+with+compacted+Topics
> >>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
> >>>>> combination.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For us it is vital as it minimized the size we had in our
> >>>>>>>>>>>>>> repartition
> >>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
> >>>>> message.
> >>>>>>>>>>>>>> I know about this new  "delete once consumer has read it".
> >>> I
> >>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> think
> >>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
> >>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
> >> but I
> >>>> am
> >>>>>>>>>>>>>> certain
> >>>>>>>>>>>>>> that you do not need RecordHeaders
> >>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
> >>> belong
> >>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> DSL
> >>>>>>>>>>>> or to the user, having a mixed use is
> >>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
> >>> don't
> >>>>>>>>>>>>>> think
> >>>>>>>>>>>>>> one
> >>>>>>>>>>>>>> needs more.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
> >>>>> couldn't
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> never
> >>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
> >>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
> >>> this
> >>>>> can
> >>>>>>>>>>>>>> also
> >>>>>>>>>>>>>> help me see the good sides a little bit more.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If there is anything unclear with all the text that has
> >> been
> >>>>>>>>>>>>>> written,
> >>>>>>>>>>>>>> feel
> >>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
> >>>>>>>>>>>>>> the mailing list.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Best Jan
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me directly
> >>>>>>>>>>>>>> previously.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
> >>>> major
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> changes
> >>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
> >> about
> >>>>> this:
> >>>>>>>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
> >>>> node
> >>>>>>>>>>>>>>> type?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Or
> >>>>>>>>>>>> would it be advisable to construct it from existing
> >> GraphNode
> >>>>>>>>>>>>>>> components?
> >>>>>>>>>>>>>>> For instance, I believe I could construct it from several
> >>>>>>>>>>>>>>> OptimizableRepartitionNode, some SinkNode, some
> >> SourceNode,
> >>>> and
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> several
> >>>>>>>>>>>> StatefulProcessorNode. That being said, there is some
> >>> underlying
> >>>>>>>>>>>>>>> complexity
> >>>>>>>>>>>>>>> to each approach.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I will be switching the KIP-213 to use the RecordHeaders
> >> in
> >>>>> Kafka
> >>>>>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>> instead of the PropagationWrapper, but conceptually it
> >>> should
> >>>>>>>>>>>>>>> be the
> >>>>>>>>>>>>>>> same.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Again, any feedback is welcomed...
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, Jul 30, 2018 at 9:38 AM, Adam Bellemare <
> >>>>>>>>>>>>>>> adam.bellemare@gmail.com
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Guozhang et al
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I was just reading the 2.0 release notes and noticed a
> >>>> section
> >>>>> on
> >>>>>>>>>>>>>>>> Record
> >>>>>>>>>>>>>>>> Headers.
> >>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>> 244%3A+Add+Record+Header+support+to+Kafka+Streams+
> >>>>> Processor+API
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I am not yet sure if the contents of a RecordHeader is
> >>>>>>>>>>>>>>>> propagated
> >>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> way through the Sinks and Sources, but if it is, and if
> >> it
> >>>>>>>>>>>>>>>> remains
> >>>>>>>>>>>>>>>> attached
> >>>>>>>>>>>>>>>> to the record (including null records) I may be able to
> >>>> ditch
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> propagationWrapper for an implementation using
> >>> RecordHeader.
> >>>>>>>>>>>>>>>> I am
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>> sure if this is doable, so if anyone understands
> >>>> RecordHeader
> >>>>>>>>>>>>>>>> impl
> >>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>> than I, I would be happy to hear from you.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> In the meantime, let me know of any questions. I believe
> >>>> this
> >>>>>>>>>>>>>>>> PR has
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>> lot
> >>>>>>>>>>>>>>>> of potential to solve problems for other people, as I
> >> have
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> encountered
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> number of other companies in the wild all home-brewing
> >>> their
> >>>>> own
> >>>>>>>>>>>>>>>> solutions
> >>>>>>>>>>>>>>>> to come up with a method of handling relational data in
> >>>>> streams.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri, Jul 27, 2018 at 1:45 AM, Guozhang
> >>>>>>>>>>>>>>>> Wang<wa...@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hello Adam,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for rebooting the discussion of this KIP ! Let me
> >>>>>>>>>>>>>>>> finish my
> >>>>>>>>>>>>>>>>> pass
> >>>>>>>>>>>>>>>>> on the wiki and get back to you soon. Sorry for the
> >>>> delays..
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Tue, Jul 24, 2018 at 6:08 AM, Adam Bellemare <
> >>>>>>>>>>>>>>>>> adam.bellemare@gmail.com
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Let me kick this off with a few starting points that I
> >>>>>>>>>>>>>>>>>> would like
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>> generate some discussion on.
> >>>>>>>>>>>>>>>>>> 1) It seems to me that I will need to repartition the
> >>> data
> >>>>>>>>>>>>>>>>>> twice -
> >>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> the foreign key, and once back to the primary key. Is
> >>>> there
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> anything
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>> missing here?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2) I believe I will also need to materialize 3 state
> >>>>>>>>>>>>>>>>>> stores: the
> >>>>>>>>>>>>>>>>>> prefixScan
> >>>>>>>>>>>>>>>>>> SS, the highwater mark SS (for out-of-order
> >> resolution)
> >>>> and
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> final
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> store, due to the workflow I have laid out. I have not
> >>>>>>>>>>>>>>>>>> thought of
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>> way yet, but would appreciate any input on this
> >> matter.
> >>> I
> >>>>> have
> >>>>>>>>>>>>>>>>>> gone
> >>>>>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>> through the mailing list for the previous discussions
> >> on
> >>>>>>>>>>>>>>>>>> this KIP,
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> did not see anything relating to resolving
> >> out-of-order
> >>>>>>>>>>>>>>>>>> compute. I
> >>>>>>>>>>>>>>>>>> cannot
> >>>>>>>>>>>>>>>>>> see a way around the current three-SS structure that I
> >>>> have.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3) Caching is disabled on the prefixScan SS, as I do
> >> not
> >>>>>>>>>>>>>>>>>> know how
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>> resolve the iterator obtained from rocksDB with that of the
> >>>> cache.
> >>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>> addition, I must ensure everything is flushed before
> >> scanning.
> >>>>>>>>>>>>>>>>> Since
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> materialized prefixScan SS is under "control" of the
> >>>>>>>>>>>>>>>>>> function, I
> >>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> anticipate this to be a problem. Performance
> >> throughput
> >>>>>>>>>>>>>>>>>> will need
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> tested, but as Jan observed in his initial overview of
> >>>> this
> >>>>>>>>>>>>>>>>>> issue,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> generally a surge of output events which affect
> >>>> performance
> >>>>>>>>>>>>>>>>>> moreso
> >>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> flush or prefixScan itself.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thoughts on any of these are greatly appreciated,
> >> since
> >>>>> these
> >>>>>>>>>>>>>>>>>> elements
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> really the cornerstone of the whole design. I can put
> >> up
> >>>>>>>>>>>>>>>>>> the code
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> written against 1.0.2 if we so desire, but first I was
> >>>>>>>>>>>>>>>>>> hoping to
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>> tackle some of the fundamental design proposals.
> >>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Mon, Jul 23, 2018 at 10:05 AM, Adam Bellemare <
> >>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Here is the new discussion thread for KIP-213. I
> >> picked
> >>>>>>>>>>>>>>>>>> back up on
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> as this is something that we too at Flipp are now
> >>> running
> >>>> in
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> production.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Jan started this last year, and I know that Trivago is
> >>>> also
> >>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> similar in production, at least in terms of APIs and
> >>>>>>>>>>>>>>>>>> functionality.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >> confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>>>> 213+Support+non-key+joining+in+KTable
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I do have an implementation of the code for Kafka
> >> 1.0.2
> >>>>> (our
> >>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>> production version) but I won't post it yet as I
> >> would
> >>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> focus
> >>>>>>>>>>>> on the
> >>>>>>>>>>>>>>>>>> workflow and design first. That being said, I also
> >> need
> >>> to
> >>>>> add
> >>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> clearer
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> integration tests (I did a lot of testing using a
> >>>> non-Kafka
> >>>>>>>>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> framework) and clean up the code a bit more before
> >>>> putting
> >>>>>>>>>>>>>>>>>>> it in
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> PR
> >>>>>>>>>>>>>>>>>>> against trunk (I can do so later this week likely).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Please take a look,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Adam Bellemare
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --
> >>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
> >
> >
> >
> > --
> > -- Guozhang
>



-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Adam,

I'd agree with the minimal approach. As far as I'm concerned, it's fine to
just implement inner join for starters.

Thanks for considering the API feedback!

Regarding #6, yes, please scrutinize it! It *seems* right to me, but it
also seems suspiciously too simple. I haven't done a proof or anything.

And regarding #7, thanks for that context.

I'd agree with testing up to 1Million on a log scale. I've seen at least
one real use case for a join fan-out of ~2M. It seems fine for large
fan-out joins to take longer; I was more concerned about the performance
when the fan-out is small (under the assumption that small fan-out is more
typical).

Of course, if you're willing to do some performance testing, we'll be able
to give people an idea of what to expect. Unless the performance is poor, I
wouldn't worry too much about tuning at this stage. And I agree with you on
the paging thing; I just wanted to throw it out there.

Anyway, the KIP is more about hashing out the public API. For something
like this, it's also useful to discuss the algorithm and implementation so
that everyone has confidence that it's actually possible to fulfill the API
contract. If it turns out that we can structure the internal nodes of the
join in some other way to improve performance, someone can do that
optimization later without even needing a KIP.

I think a clean API and a simple implementation with reasonable performance
is a good starting place.

Finally, yes, we should probably hear from some committers on the current
iteration of the KIP before proceeding with the vote.

Thanks for your consideration,
-John


On Fri, Nov 30, 2018 at 9:24 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi John
>
> Thanks for the feedback. It is extremely valuable. Responses as follows:
>
> 1. I could definitely override the join function.
>
> The API you proposed is fine by me:
> KTable<K, VR> join(KTable<KO, VO> other,
>                    ValueMapper<V, KO> foreignKeyExtractor,
>                    ValueJoiner<V, VO, VR> joiner,
>                    ManyToOneJoined manyToOneJoined, // join config object
>                   );
>
> 2. Currently this is only an inner-join. The join logic is only executed
> when records exist in both KTables. It could be expanded to do other joins
> too, but I wanted to limit the scope in the first pass and focus on a
> minimum product. If this is accepted, I could generate a ticket to work on
> the outer and left join.
>
> 3a. True. I will revisit the input requirements of the function.
>
> 3b. It is only necessary if the user has custom-partitioned this or other
> KTable. The current implementation assumes that it's the default hash if
> nothing else is specified. However, in line with the other joins, it would
> make sense to assume they are identically partitioned. I can remove it no
> problem.
>
> 4. No longer relevant. See 6.
>
> 5. No longer relevant. See 6.
>
> 6. Brilliant! Thank you for this piece of feedback. This is me missing the
> forest for the trees. I failed to remember that the events would indeed end
> up back at their original node, and could be compared in such a way. I have
> implemented this solution this morning and it seems to work. I will mull it
> over a bit more to see if I can find any problematic race conditions.
> Provided this is indeed sound, I believe you neatly sliced this problem
> down to a much neater and simpler size.
>
> 7. We have been using this in production. The scale of data is generally
> around the many being in the 100-500 entity range. Sometimes we do have
> 10000+ in the "many" and indeed it does take a bit longer. I suspect it
> would be best to measure it directly, something like:
>
> - 1 to 100 through 1 Million in a factor of 10
> - Reduce RocksDB in-memory allocation to a minimal amount to exercise disk
> seeks.
>
> I am not sure how far down the path to go with regards to testing RocksDB
> performance vs. Kafka Streams functionality, but I will take a look and get
> back to you on this.
>
> With regards to changing the structure to an unbounded value size, in my
> use-cases we have already run into the size problem and I would prefer to
> avoid a paging solution. I will focus on finding a means to performance
> test before making any more design changes.
>
> Thanks again John, I am very appreciative. I suspect this may need to move
> back to DISCUSS though (which I am fine with)
>
> Adam
>
>
>
>
>
>
> On Thu, Nov 29, 2018 at 4:18 PM John Roesler <jo...@confluent.io> wrote:
>
> > Actually, I've been thinking more about my feedback #1 on the method
> name,
> > and I'm not so sure it's a good idea.
> >
> > Following SQL, the existing join methods are named according to
> handedness
> > (inner/outer/left). All these joins are the same cardinality (1:1
> joins). I
> > think it would be a mistake to switch up the naming scheme and introduce
> a
> > new method named for cardinality, like "manyToOneJoin".
> >
> > Can we actually just add a new overload to the existing joins? The
> javadoc
> > could explain that it's a many-to-one join, and it would be
> differentiated
> > by the presence of the "keyExtractor".
> >
> > Just to be safe, if we rename the proposed "joinOnForeignKey" to just
> > "join", then we could rename "keyExtractor" to "foreignKeyExtractor" for
> > clarity.
> >
> > Just to be unambiguous, including the other API feedback I gave, I'm
> > proposing something like this:
> >
> > KTable<K, VR> join(KTable<KO, VO> other,
> >                    ValueMapper<V, KO> foreignKeyExtractor,
> >                    ValueJoiner<V, VO, VR> joiner,
> >                    ManyToOneJoined manyToOneJoined, // join config object
> >                   );
> >
> > The ManyToOneJoined config object would allow setting all 4 serdes and
> > configuring materialization.
> >
> > Thanks for your consideration,
> > -John
> >
> > On Thu, Nov 29, 2018 at 8:14 AM John Roesler <jo...@confluent.io> wrote:
> >
> > > Hi all,
> > >
> > > Sorry that this discussion petered out... I think the 2.1 release
> caused
> > > an extended distraction that pushed it off everyone's radar (which was
> > > precisely Adam's concern). Personally, I've also had some extend
> > > distractions of my own that kept (and continue to keep) me preoccupied.
> > >
> > > However, calling for a vote did wake me up, so I guess Jan was on the
> > > right track!
> > >
> > > I've gone back and reviewed the whole KIP document and the prior
> > > discussion, and I'd like to offer a few thoughts:
> > >
> > > API Thoughts:
> > >
> > > 1. If I read the KIP right, you are proposing a many-to-one join. Could
> > we
> > > consider naming it manyToOneJoin? Or, if you prefer, flip the design
> > around
> > > and make it a oneToManyJoin?
> > >
> > > The proposed name "joinOnForeignKey" disguises the join type, and it
> > seems
> > > like it might trick some people into using it for a one-to-one join.
> This
> > > would work, of course, but it would be super inefficient compared to a
> > > simple rekey-and-join.
> > >
> > > 2. I might have missed it, but I don't think it's specified whether
> it's
> > > an inner, outer, or left join. I'm guessing an outer join, as
> (neglecting
> > > IQ), the rest can be achieved by filtering or by handling it in the
> > > ValueJoiner.
> > >
> > > 3. The arg list to joinOnForeignKey doesn't look quite right.
> > > 3a. Regarding Serialized: There are a few different paradigms in play
> in
> > > the Streams API, so it's confusing, but instead of three Serialized
> > args, I
> > > think it would be better to have one that allows (optionally) setting
> > the 4
> > > incoming serdes. The result serde is defined by the Materialized. The
> > > incoming serdes can be optional because they might already be available
> > on
> > > the source KTables, or the default serdes from the config might be
> > > applicable.
> > >
> > > 3b. Is the StreamPartitioner necessary? The other joins don't allow
> > > setting one, and it seems like it might actually be harmful, since the
> > > rekey operation needs to produce results that are co-partitioned with
> the
> > > "other" KTable.
> > >
> > > 4. I'm fine with the "reserved word" header, but I didn't actually
> follow
> > > what Matthias meant about namespacing requiring "deserializing" the
> > record
> > > header. The headers are already Strings, so I don't think that
> > > deserialization is required. If we applied the namespace at source
> nodes
> > > and stripped it at sink nodes, this would be practically no overhead.
> The
> > > advantage of the namespace idea is that no public API change wrt
> headers
> > > needs to happen, and no restrictions need to be placed on users'
> headers.
> > >
> > > (Although I'm wondering if we can get away without the header at all...
> > > stay tuned)
> > >
> > > 5. I also didn't follow the discussion about the HWM table growing
> > without
> > > bound. As I read it, the HWM table is effectively implementing OCC to
> > > resolve the problem you noted with disordering when the rekey is
> > > reversed... particularly notable when the FK changes. As such, it only
> > > needs to track the most recent "version" (the offset in the source
> > > partition) of each key. Therefore, it should have the same number of
> keys
> > > as the source table at all times.
> > >
> > > I see that you are aware of KIP-258, which I think might be relevant
> in a
> > > couple of ways. One: it's just about storing the timestamp in the state
> > > store, but the ultimate idea is to effectively use the timestamp as an
> > OCC
> > > "version" to drop disordered updates. You wouldn't want to use the
> > > timestamp for this operation, but if you were to use a similar
> mechanism
> > to
> > > store the source offset in the store alongside the re-keyed values,
> then
> > > you could avoid a separate table.
> > >
> > > 6. You and Jan have been thinking about this for a long time, so I've
> > > probably missed something here, but I'm wondering if we can avoid the
> HWM
> > > tracking at all and resolve out-of-order during a final join instead...
> > >
> > > Let's say we're joining a left table (Integer K: Letter FK, (other
> data))
> > > to a right table (Letter K: (some data)).
> > >
> > > Left table:
> > > 1: (A, xyz)
> > > 2: (B, asd)
> > >
> > > Right table:
> > > A: EntityA
> > > B: EntityB
> > >
> > > We could do a rekey as you proposed with a combined key, but not
> > > propagating the value at all..
> > > Rekey table:
> > > A-1: (dummy value)
> > > B-2: (dummy value)
> > >
> > > Which we then join with the right table to produce:
> > > A-1: EntityA
> > > B-2: EntityB
> > >
> > > Which gets rekeyed back:
> > > 1: A, EntityA
> > > 2: B, EntityB
> > >
> > > And finally we do the actual join:
> > > Result table:
> > > 1: ((A, xyz), EntityA)
> > > 2: ((B, asd), EntityB)
> > >
> > > The thing is that in that last join, we have the opportunity to compare
> > > the current FK in the left table with the incoming PK of the right
> table.
> > > If they don't match, we just drop the event, since it must be outdated.
> > >
> > > In your KIP, you gave an example in which (1: A, xyz) gets updated to
> (1:
> > > B, xyz), ultimately yielding a conundrum about whether the final state
> > > should be (1: null) or (1: joined-on-B). With the algorithm above, you
> > > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> > > EntityB)). It seems like this does give you enough information to make
> > the
> > > right choice, regardless of disordering.
> > >
> > >
> > > 7. Last thought... I'm a little concerned about the performance of the
> > > range scans when records change in the right table. You've said that
> > you've
> > > been using the algorithm you presented in production for a while. Can
> you
> > > give us a sense of the performance characteristics you've observed?
> > >
> > > I could only think of one alternative, but I'm not sure if it's better
> or
> > > worse... If the first re-key only needs to preserve the original key,
> as
> > I
> > > proposed in #6, then we could store a vector of keys in the value:
> > >
> > > Left table:
> > > 1: A,...
> > > 2: B,...
> > > 3: A,...
> > >
> > > Gets re-keyed:
> > > A: [1, 3]
> > > B: [2]
> > >
> > > Then, the rhs part of the join would only need a regular single-key
> > > lookup. Of course we have to deal with the problem of large values, as
> > > there's no bound on the number of lhs records that can reference rhs
> > > records. Offhand, I'd say we could page the values, so when one row is
> > past
> > > the threshold, we append the key for the next page. Then in most cases,
> > it
> > > would be a single key lookup, but for large fan-out updates, it would
> be
> > > one per (max value size)/(avg lhs key size).
> > >
> > > This seems more complex, though... Plus, I think there's some extra
> > > tracking we'd need to do to know when to emit a retraction. For
> example,
> > > when record 1 is deleted, the re-key table would just have (A: [3]).
> Some
> > > kind of tombstone is needed so that the join result for 1 can also be
> > > retracted.
> > >
> > > That's all!
> > >
> > > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> > > discussion has been slow.
> > > -John
> > >
> > >
> > > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Jan.Filipiak@trivago.com
> >
> > > wrote:
> > >
> > >> Id say you can just call the vote.
> > >>
> > >> that happens all the time, and if something comes up, it just goes
> back
> > >> to discuss.
> > >>
> > >> would not expect to much attention with another another email in this
> > >> thread.
> > >>
> > >> best Jan
> > >>
> > >> On 09.10.2018 13:56, Adam Bellemare wrote:
> > >> > Hello Contributors
> > >> >
> > >> > I know that 2.1 is about to be released, but I do need to bump this
> to
> > >> keep
> > >> > visibility up. I am still intending to push this through once
> > >> contributor
> > >> > feedback is given.
> > >> >
> > >> > Main points that need addressing:
> > >> > 1) Any way (or benefit) in structuring the current singular graph
> node
> > >> into
> > >> > multiple nodes? It has a whopping 25 parameters right now. I am a
> bit
> > >> fuzzy
> > >> > on how the optimizations are supposed to work, so I would appreciate
> > any
> > >> > help on this aspect.
> > >> >
> > >> > 2) Overall strategy for joining + resolving. This thread has much
> > >> discourse
> > >> > between Jan and I between the current highwater mark proposal and a
> > >> groupBy
> > >> > + reduce proposal. I am of the opinion that we need to strictly
> handle
> > >> any
> > >> > chance of out-of-order data and leave none of it up to the consumer.
> > Any
> > >> > comments or suggestions here would also help.
> > >> >
> > >> > 3) Anything else that you see that would prevent this from moving
> to a
> > >> vote?
> > >> >
> > >> > Thanks
> > >> >
> > >> > Adam
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > >> adam.bellemare@gmail.com>
> > >> > wrote:
> > >> >
> > >> >> Hi Jan
> > >> >>
> > >> >> With the Stores.windowStoreBuilder and
> Stores.persistentWindowStore,
> > >> you
> > >> >> actually only need to specify the amount of segments you want and
> how
> > >> large
> > >> >> they are. To the best of my understanding, what happens is that the
> > >> >> segments are automatically rolled over as new data with new
> > timestamps
> > >> are
> > >> >> created. We use this exact functionality in some of the work done
> > >> >> internally at my company. For reference, this is the hopping
> windowed
> > >> store.
> > >> >>
> > >> >>
> > >>
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > >> >>
> > >> >> In the code that I have provided, there are going to be two 24h
> > >> segments.
> > >> >> When a record is put into the windowStore, it will be inserted at
> > time
> > >> T in
> > >> >> both segments. The two segments will always overlap by 12h. As time
> > >> goes on
> > >> >> and new records are added (say at time T+12h+), the oldest segment
> > >> will be
> > >> >> automatically deleted and a new segment created. The records are by
> > >> default
> > >> >> inserted with the context.timestamp(), such that it is the record
> > >> time, not
> > >> >> the clock time, which is used.
> > >> >>
> > >> >> To the best of my understanding, the timestamps are retained when
> > >> >> restoring from the changelog.
> > >> >>
> > >> >> Basically, this is heavy-handed way to deal with TTL at a
> > >> segment-level,
> > >> >> instead of at an individual record level.
> > >> >>
> > >> >> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > Jan.Filipiak@trivago.com
> > >> >
> > >> >> wrote:
> > >> >>
> > >> >>> Will that work? I expected it to blow up with ClassCastException
> or
> > >> >>> similar.
> > >> >>>
> > >> >>> You either would have to specify the window you fetch/put or
> iterate
> > >> >>> across all windows the key was found in right?
> > >> >>>
> > >> >>> I just hope the window-store doesn't check stream-time under the
> > hoods
> > >> >>> that would be a questionable interface.
> > >> >>>
> > >> >>> If it does: did you see my comment on checking all the windows
> > >> earlier?
> > >> >>> that would be needed to actually give reasonable time gurantees.
> > >> >>>
> > >> >>> Best
> > >> >>>
> > >> >>>
> > >> >>>
> > >> >>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > >> >>>> Hi Jan
> > >> >>>>
> > >> >>>> Check for  " highwaterMat " in the PR. I only changed the state
> > >> store,
> > >> >>> not
> > >> >>>> the ProcessorSupplier.
> > >> >>>>
> > >> >>>> Thanks,
> > >> >>>> Adam
> > >> >>>>
> > >> >>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > >> Jan.Filipiak@trivago.com
> > >> >>>>
> > >> >>>> wrote:
> > >> >>>>
> > >> >>>>>
> > >> >>>>>
> > >> >>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > >> >>>>>
> > >> >>>>>> @Guozhang
> > >> >>>>>>
> > >> >>>>>> Thanks for the information. This is indeed something that will
> be
> > >> >>>>>> extremely
> > >> >>>>>> useful for this KIP.
> > >> >>>>>>
> > >> >>>>>> @Jan
> > >> >>>>>> Thanks for your explanations. That being said, I will not be
> > moving
> > >> >>> ahead
> > >> >>>>>> with an implementation using reshuffle/groupBy solution as you
> > >> >>> propose.
> > >> >>>>>> That being said, if you wish to implement it yourself off of my
> > >> >>> current PR
> > >> >>>>>> and submit it as a competitive alternative, I would be more
> than
> > >> >>> happy to
> > >> >>>>>> help vet that as an alternate solution. As it stands right
> now, I
> > >> do
> > >> >>> not
> > >> >>>>>> really have more time to invest into alternatives without there
> > >> being
> > >> >>> a
> > >> >>>>>> strong indication from the binding voters which they would
> > prefer.
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>> Hey, total no worries. I think I personally gave up on the
> streams
> > >> DSL
> > >> >>> for
> > >> >>>>> some time already, otherwise I would have pulled this KIP
> through
> > >> >>> already.
> > >> >>>>> I am currently reimplementing my own DSL based on PAPI.
> > >> >>>>>
> > >> >>>>>
> > >> >>>>>> I will look at finishing up my PR with the windowed state store
> > in
> > >> the
> > >> >>>>>> next
> > >> >>>>>> week or so, exercising it via tests, and then I will come back
> > for
> > >> >>> final
> > >> >>>>>> discussions. In the meantime, I hope that any of the binding
> > voters
> > >> >>> could
> > >> >>>>>> take a look at the KIP in the wiki. I have updated it according
> > to
> > >> the
> > >> >>>>>> latest plan:
> > >> >>>>>>
> > >> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > >> >>>>>> Support+non-key+joining+in+KTable
> > >> >>>>>>
> > >> >>>>>> I have also updated the KIP PR to use a windowed store. This
> > could
> > >> be
> > >> >>>>>> replaced by the results of KIP-258 whenever they are completed.
> > >> >>>>>> https://github.com/apache/kafka/pull/5527
> > >> >>>>>>
> > >> >>>>>> Thanks,
> > >> >>>>>>
> > >> >>>>>> Adam
> > >> >>>>>>
> > >> >>>>>
> > >> >>>>> Is the HighWatermarkResolverProccessorsupplier already updated
> in
> > >> the
> > >> >>> PR?
> > >> >>>>> expected it to change to Windowed<K>,Long Missing something?
> > >> >>>>>
> > >> >>>>>
> > >> >>>>>
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > >> >
> > >> >>>>>> wrote:
> > >> >>>>>>
> > >> >>>>>> Correction on my previous email: KAFKA-5533 is the wrong link,
> as
> > >> it
> > >> >>> is
> > >> >>>>>>> for
> > >> >>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we
> do
> > >> >>> want to
> > >> >>>>>>> have "handling out-of-order data for source KTable" such that
> > >> >>> instead of
> > >> >>>>>>> blindly apply the updates to the materialized store, i.e.
> > >> following
> > >> >>>>>>> offset
> > >> >>>>>>> ordering, we will reject updates that are older than the
> current
> > >> >>> key's
> > >> >>>>>>> timestamps, i.e. following timestamp ordering.
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> Guozhang
> > >> >>>>>>>
> > >> >>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> > >> wangguoz@gmail.com>
> > >> >>>>>>> wrote:
> > >> >>>>>>>
> > >> >>>>>>> Hello Adam,
> > >> >>>>>>>>
> > >> >>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
> the
> > >> high
> > >> >>>>>>>> watermark store, now altered to be replaced with a window
> > >> store), I
> > >> >>>>>>>> think
> > >> >>>>>>>> another current on-going KIP may actually help:
> > >> >>>>>>>>
> > >> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> This is for adding the timestamp into a key-value store (i.e.
> > >> only
> > >> >>> for
> > >> >>>>>>>> non-windowed KTable), and then one of its usage, as described
> > in
> > >> >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we
> > can
> > >> >>> then
> > >> >>>>>>>> "reject" updates from the source topics if its timestamp is
> > >> smaller
> > >> >>> than
> > >> >>>>>>>> the current key's latest update timestamp. I think it is very
> > >> >>> similar to
> > >> >>>>>>>> what you have in mind for high watermark based filtering,
> while
> > >> you
> > >> >>> only
> > >> >>>>>>>> need to make sure that the timestamps of the joining records
> > are
> > >> >>>>>>>>
> > >> >>>>>>> correctly
> > >> >>>>>>>
> > >> >>>>>>>> inherited though the whole topology to the final stage.
> > >> >>>>>>>>
> > >> >>>>>>>> Note that this KIP is for key-value store and hence
> > non-windowed
> > >> >>> KTables
> > >> >>>>>>>> only, but for windowed KTables we do not really have a good
> > >> support
> > >> >>> for
> > >> >>>>>>>> their joins anyways (
> > >> >>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > >> >>>>>>>> I
> > >> >>>>>>>> think we can just consider non-windowed KTable-KTable non-key
> > >> joins
> > >> >>> for
> > >> >>>>>>>> now. In which case, KIP-258 should help.
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> Guozhang
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > >> >>> Jan.Filipiak@trivago.com
> > >> >>>>>>>>>
> > >> >>>>>>>> wrote:
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > >> >>>>>>>>>
> > >> >>>>>>>>> Hi Guozhang
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Current highwater mark implementation would grow endlessly
> > >> based
> > >> >>> on
> > >> >>>>>>>>>> primary key of original event. It is a pair of (<this table
> > >> >>> primary
> > >> >>>>>>>>>>
> > >> >>>>>>>>> key>,
> > >> >>>>>>>
> > >> >>>>>>>> <highest offset seen for that key>). This is used to
> > >> differentiate
> > >> >>>>>>>>>>
> > >> >>>>>>>>> between
> > >> >>>>>>>
> > >> >>>>>>>> late arrivals and new updates. My newest proposal would be to
> > >> >>> replace
> > >> >>>>>>>>>>
> > >> >>>>>>>>> it
> > >> >>>>>>>
> > >> >>>>>>>> with a Windowed state store of Duration N. This would allow
> the
> > >> same
> > >> >>>>>>>>>> behaviour, but cap the size based on time. This should
> allow
> > >> for
> > >> >>> all
> > >> >>>>>>>>>> late-arriving events to be processed, and should be
> > >> customizable
> > >> >>> by
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
> > minutes
> > >> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>> window,
> > >> >>>>>>>
> > >> >>>>>>>> or perhaps 7 days...).
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Adam, using time based retention can do the trick here.
> > Even
> > >> >>> if I
> > >> >>>>>>>>> would still like to see the automatic repartitioning
> optional
> > >> >>> since I
> > >> >>>>>>>>>
> > >> >>>>>>>> would
> > >> >>>>>>>
> > >> >>>>>>>> just reshuffle again. With windowed store I am a little bit
> > >> >>> sceptical
> > >> >>>>>>>>>
> > >> >>>>>>>> about
> > >> >>>>>>>
> > >> >>>>>>>> how to determine the window. So esentially one could run into
> > >> >>> problems
> > >> >>>>>>>>>
> > >> >>>>>>>> when
> > >> >>>>>>>
> > >> >>>>>>>> the rapid change happens near a window border. I will check
> you
> > >> >>>>>>>>> implementation in detail, if its problematic, we could still
> > >> check
> > >> >>>>>>>>> _all_
> > >> >>>>>>>>> windows on read with not to bad performance impact I guess.
> > Will
> > >> >>> let
> > >> >>>>>>>>> you
> > >> >>>>>>>>> know if the implementation would be correct as is. I
> wouldn't
> > >> not
> > >> >>> like
> > >> >>>>>>>>>
> > >> >>>>>>>> to
> > >> >>>>>>>
> > >> >>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> > >> timestamp(B).
> > >> >>> I
> > >> >>>>>>>>>
> > >> >>>>>>>> think
> > >> >>>>>>>
> > >> >>>>>>>> we can't expect that.
> > >> >>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> @Jan
> > >> >>>>>>>>>> I believe I understand what you mean now - thanks for the
> > >> >>> diagram, it
> > >> >>>>>>>>>> did really help. You are correct that I do not have the
> > >> original
> > >> >>>>>>>>>>
> > >> >>>>>>>>> primary
> > >> >>>>>>>
> > >> >>>>>>>> key available, and I can see that if it was available then
> you
> > >> >>> would be
> > >> >>>>>>>>>> able to add and remove events from the Map. That being
> said,
> > I
> > >> >>>>>>>>>>
> > >> >>>>>>>>> encourage
> > >> >>>>>>>
> > >> >>>>>>>> you to finish your diagrams / charts just for clarity for
> > >> everyone
> > >> >>>>>>>>>>
> > >> >>>>>>>>> else.
> > >> >>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> > >> >>> understand
> > >> >>>>>>>>>>
> > >> >>>>>>>>> the benefits for the rest. Sorry about the original primary
> > >> key, We
> > >> >>>>>>>>> have
> > >> >>>>>>>>> join and Group by implemented our own in PAPI and basically
> > not
> > >> >>> using
> > >> >>>>>>>>>
> > >> >>>>>>>> any
> > >> >>>>>>>
> > >> >>>>>>>> DSL (Just the abstraction). Completely missed that in
> original
> > >> DSL
> > >> >>> its
> > >> >>>>>>>>>
> > >> >>>>>>>> not
> > >> >>>>>>>
> > >> >>>>>>>> there and just assumed it. total brain mess up on my end.
> Will
> > >> >>> finish
> > >> >>>>>>>>>
> > >> >>>>>>>> the
> > >> >>>>>>>
> > >> >>>>>>>> chart as soon as i get a quite evening this week.
> > >> >>>>>>>>>
> > >> >>>>>>>>> My follow up question for you is, won't the Map stay inside
> > the
> > >> >>> State
> > >> >>>>>>>>>
> > >> >>>>>>>>>> Store indefinitely after all of the changes have
> propagated?
> > >> Isn't
> > >> >>>>>>>>>> this
> > >> >>>>>>>>>> effectively the same as a highwater mark state store?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thing is that if the map is empty, substractor is gonna
> > return
> > >> >>> `null`
> > >> >>>>>>>>>
> > >> >>>>>>>> and
> > >> >>>>>>>
> > >> >>>>>>>> the key is removed from the keyspace. But there is going to
> be
> > a
> > >> >>> store
> > >> >>>>>>>>> 100%, the good thing is that I can use this store directly
> for
> > >> >>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
> > >> >>> satisfying
> > >> >>>>>>>>> all gurantees needed for further groupby / join. The
> Windowed
> > >> >>> store is
> > >> >>>>>>>>>
> > >> >>>>>>>> not
> > >> >>>>>>>
> > >> >>>>>>>> keeping the values, so for the next statefull operation we
> > would
> > >> >>>>>>>>> need to instantiate an extra store. or we have the window
> > store
> > >> >>> also
> > >> >>>>>>>>>
> > >> >>>>>>>> have
> > >> >>>>>>>
> > >> >>>>>>>> the values then.
> > >> >>>>>>>>>
> > >> >>>>>>>>> Long story short. if we can flip in a custom group by before
> > >> >>>>>>>>> repartitioning to the original primary key i think it would
> > help
> > >> >>> the
> > >> >>>>>>>>>
> > >> >>>>>>>> users
> > >> >>>>>>>
> > >> >>>>>>>> big time in building efficient apps. Given the original
> primary
> > >> key
> > >> >>>>>>>>>
> > >> >>>>>>>> issue I
> > >> >>>>>>>
> > >> >>>>>>>> understand that we do not have a solid foundation to build
> on.
> > >> >>>>>>>>> Leaving primary key carry along to the user. very
> > unfortunate. I
> > >> >>> could
> > >> >>>>>>>>> understand the decision goes like that. I do not think its a
> > >> good
> > >> >>>>>>>>>
> > >> >>>>>>>> decision.
> > >> >>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks
> > >> >>>>>>>>>> Adam
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> > >> >>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > >> dumbreprajakta311@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>> wrote:
> > >> >>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>>>        please remove me from this group
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> > >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> > >> Jan.Filipiak@trivago.com
> > >> >>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > Hi Adam,
> > >> >>>>>>>>>>        >
> > >> >>>>>>>>>>        > give me some time, will make such a chart. last
> > time i
> > >> >>> didn't
> > >> >>>>>>>>>>        get along
> > >> >>>>>>>>>>        > well with giphy and ruined all your charts.
> > >> >>>>>>>>>>        > Hopefully i can get it done today
> > >> >>>>>>>>>>        >
> > >> >>>>>>>>>>        > On 08.09.2018 16:00, Adam Bellemare wrote:
> > >> >>>>>>>>>>        > > Hi Jan
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > > I have included a diagram of what I attempted on
> > the
> > >> >>> KIP.
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        >
> > >> >>>>>>>>>>
> > >> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > >> >>>>>>>>>>
> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > >> >>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > >> >>>>>>>>>>        <
> > >> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > >> >>>>>>>>>>
> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > >> >>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > > I attempted this back at the start of my own
> > >> >>> implementation
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>        this
> > >> >>>>>>>>>>        > > solution, and since I could not get it to work I
> > >> have
> > >> >>> since
> > >> >>>>>>>>>>        discarded the
> > >> >>>>>>>>>>        > > code. At this point in time, if you wish to
> > continue
> > >> >>> pursuing
> > >> >>>>>>>>>>        for your
> > >> >>>>>>>>>>        > > groupBy solution, I ask that you please create a
> > >> >>> diagram on
> > >> >>>>>>>>>>        the KIP
> > >> >>>>>>>>>>        > > carefully explaining your solution. Please feel
> > >> free to
> > >> >>> use
> > >> >>>>>>>>>>        the image I
> > >> >>>>>>>>>>        > > just posted as a starting point. I am having
> > trouble
> > >> >>>>>>>>>>        understanding your
> > >> >>>>>>>>>>        > > explanations but I think that a carefully
> > >> constructed
> > >> >>> diagram
> > >> >>>>>>>>>>        will clear
> > >> >>>>>>>>>>        > up
> > >> >>>>>>>>>>        > > any misunderstandings. Alternately, please post
> a
> > >> >>>>>>>>>>        comprehensive PR with
> > >> >>>>>>>>>>        > > your solution. I can only guess at what you
> mean,
> > >> and
> > >> >>> since I
> > >> >>>>>>>>>>        value my
> > >> >>>>>>>>>>        > own
> > >> >>>>>>>>>>        > > time as much as you value yours, I believe it is
> > >> your
> > >> >>>>>>>>>>        responsibility to
> > >> >>>>>>>>>>        > > provide an implementation instead of me trying
> to
> > >> guess.
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > > Adam
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> > >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> > >> Jan.Filipiak@trivago.com
> > >> >>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > > wrote:
> > >> >>>>>>>>>>        > >
> > >> >>>>>>>>>>        > >> Hi James,
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >> nice to see you beeing interested. kafka
> streams
> > at
> > >> >>> this
> > >> >>>>>>>>>>        point supports
> > >> >>>>>>>>>>        > >> all sorts of joins as long as both streams have
> > the
> > >> >>> same
> > >> >>>>>>>>>> key.
> > >> >>>>>>>>>>        > >> Adam is currently implementing a join where a
> > >> KTable
> > >> >>> and a
> > >> >>>>>>>>>>        KTable can
> > >> >>>>>>>>>>        > have
> > >> >>>>>>>>>>        > >> a one to many relation ship (1:n). We exploit
> > that
> > >> >>> rocksdb
> > >> >>>>>>>>>> is
> > >> >>>>>>>>>>
> > >> >>>>>>>>> a
> > >> >>>>>>>
> > >> >>>>>>>>        > >> datastore that keeps data sorted (At least
> exposes
> > an
> > >> >>> API to
> > >> >>>>>>>>>>        access the
> > >> >>>>>>>>>>        > >> stored data in a sorted fashion).
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >> I think the technical caveats are well
> understood
> > >> now
> > >> >>> and we
> > >> >>>>>>>>>>
> > >> >>>>>>>>> are
> > >> >>>>>>>
> > >> >>>>>>>>        > basically
> > >> >>>>>>>>>>        > >> down to philosophy and API Design ( when Adam
> > sees
> > >> my
> > >> >>> newest
> > >> >>>>>>>>>>        message).
> > >> >>>>>>>>>>        > >> I have a lengthy track record of loosing those
> > >> kinda
> > >> >>>>>>>>>>        arguments within
> > >> >>>>>>>>>>        > the
> > >> >>>>>>>>>>        > >> streams community and I have no clue why. So I
> > >> >>> literally
> > >> >>>>>>>>>>        can't wait for
> > >> >>>>>>>>>>        > you
> > >> >>>>>>>>>>        > >> to churn through this thread and give you
> opinion
> > >> on
> > >> >>> how we
> > >> >>>>>>>>>>        should
> > >> >>>>>>>>>>        > design
> > >> >>>>>>>>>>        > >> the return type of the oneToManyJoin and how
> many
> > >> >>> power we
> > >> >>>>>>>>>>        want to give
> > >> >>>>>>>>>>        > to
> > >> >>>>>>>>>>        > >> the user vs "simplicity" (where simplicity
> isn't
> > >> >>> really that
> > >> >>>>>>>>>>        as users
> > >> >>>>>>>>>>        > still
> > >> >>>>>>>>>>        > >> need to understand it I argue)
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >> waiting for you to join in on the discussion
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >> Best Jan
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >> On 07.09.2018 15:49, James Kwan wrote:
> > >> >>>>>>>>>>        > >>
> > >> >>>>>>>>>>        > >>> I am new to this group and I found this
> subject
> > >> >>>>>>>>>>        interesting.  Sounds
> > >> >>>>>>>>>>        > like
> > >> >>>>>>>>>>        > >>> you guys want to implement a join table of two
> > >> >>> streams? Is
> > >> >>>>>>>>>> there
> > >> >>>>>>>>>>        > somewhere
> > >> >>>>>>>>>>        > >>> I can see the original requirement or
> proposal?
> > >> >>>>>>>>>>        > >>>
> > >> >>>>>>>>>>        > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> > >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> > >> Jan.Filipiak@trivago.com
> > >> >>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > >>>> wrote:
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>> I'm currently testing using a Windowed Store
> > to
> > >> >>> store the
> > >> >>>>>>>>>>        highwater
> > >> >>>>>>>>>>        > >>>>> mark.
> > >> >>>>>>>>>>        > >>>>> By all indications this should work fine,
> with
> > >> the
> > >> >>> caveat
> > >> >>>>>>>>>>        being that
> > >> >>>>>>>>>>        > it
> > >> >>>>>>>>>>        > >>>>> can
> > >> >>>>>>>>>>        > >>>>> only resolve out-of-order arrival for up to
> > the
> > >> >>> size of
> > >> >>>>>>>>>>        the window
> > >> >>>>>>>>>>        > (ie:
> > >> >>>>>>>>>>        > >>>>> 24h, 72h, etc). This would remove the
> > >> possibility
> > >> >>> of it
> > >> >>>>>>>>>>
> > >> >>>>>>>>> being
> > >> >>>>>>>
> > >> >>>>>>>>        > unbounded
> > >> >>>>>>>>>>        > >>>>> in
> > >> >>>>>>>>>>        > >>>>> size.
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>> With regards to Jan's suggestion, I believe
> > >> this is
> > >> >>> where
> > >> >>>>>>>>>>        we will
> > >> >>>>>>>>>>        > have
> > >> >>>>>>>>>>        > >>>>> to
> > >> >>>>>>>>>>        > >>>>> remain in disagreement. While I do not
> > disagree
> > >> >>> with your
> > >> >>>>>>>>>>        statement
> > >> >>>>>>>>>>        > >>>>> about
> > >> >>>>>>>>>>        > >>>>> there likely to be additional joins done in
> a
> > >> >>> real-world
> > >> >>>>>>>>>>        workflow, I
> > >> >>>>>>>>>>        > do
> > >> >>>>>>>>>>        > >>>>> not
> > >> >>>>>>>>>>        > >>>>> see how you can conclusively deal with
> > >> out-of-order
> > >> >>>>>>>>>> arrival
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>        > >>>>> foreign-key
> > >> >>>>>>>>>>        > >>>>> changes and subsequent joins. I have
> attempted
> > >> what
> > >> >>> I
> > >> >>>>>>>>>>        think you have
> > >> >>>>>>>>>>        > >>>>> proposed (without a high-water, using
> groupBy
> > >> and
> > >> >>> reduce)
> > >> >>>>>>>>>>        and found
> > >> >>>>>>>>>>        > >>>>> that if
> > >> >>>>>>>>>>        > >>>>> the foreign key changes too quickly, or the
> > >> load on
> > >> >>> a
> > >> >>>>>>>>>>        stream thread
> > >> >>>>>>>>>>        > is
> > >> >>>>>>>>>>        > >>>>> too
> > >> >>>>>>>>>>        > >>>>> high, the joined messages will arrive
> > >> out-of-order
> > >> >>> and be
> > >> >>>>>>>>>>        incorrectly
> > >> >>>>>>>>>>        > >>>>> propagated, such that an intermediate event
> is
> > >> >>>>>>>>>> represented
> > >> >>>>>>>>>>        as the
> > >> >>>>>>>>>>        > final
> > >> >>>>>>>>>>        > >>>>> event.
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>> Can you shed some light on your groupBy
> > >> >>> implementation.
> > >> >>>>>>>>>>        There must be
> > >> >>>>>>>>>>        > >>>> some sort of flaw in it.
> > >> >>>>>>>>>>        > >>>> I have a suspicion where it is, I would just
> > >> like to
> > >> >>>>>>>>>>        confirm. The idea
> > >> >>>>>>>>>>        > >>>> is bullet proof and it must be
> > >> >>>>>>>>>>        > >>>> an implementation mess up. I would like to
> > >> clarify
> > >> >>> before
> > >> >>>>>>>>>>        we draw a
> > >> >>>>>>>>>>        > >>>> conclusion.
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>    Repartitioning the scattered events back
> to
> > >> their
> > >> >>>>>>>>>>
> > >> >>>>>>>>> original
> > >> >>>>>>>
> > >> >>>>>>>>        > >>>>> partitions is the only way I know how to
> > >> conclusively
> > >> >>> deal
> > >> >>>>>>>>>>        with
> > >> >>>>>>>>>>        > >>>>> out-of-order events in a given time frame,
> and
> > >> to
> > >> >>> ensure
> > >> >>>>>>>>>>        that the
> > >> >>>>>>>>>>        > data
> > >> >>>>>>>>>>        > >>>>> is
> > >> >>>>>>>>>>        > >>>>> eventually consistent with the input events.
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>> If you have some code to share that
> > illustrates
> > >> your
> > >> >>>>>>>>>>        approach, I
> > >> >>>>>>>>>>        > would
> > >> >>>>>>>>>>        > >>>>> be
> > >> >>>>>>>>>>        > >>>>> very grateful as it would remove any
> > >> >>> misunderstandings
> > >> >>>>>>>>>>        that I may
> > >> >>>>>>>>>>        > have.
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>> ah okay you were looking for my code. I don't
> > >> have
> > >> >>>>>>>>>>        something easily
> > >> >>>>>>>>>>        > >>>> readable here as its bloated with
> OO-patterns.
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>> its anyhow trivial:
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>> @Override
> > >> >>>>>>>>>>        > >>>>      public T apply(K aggKey, V value, T
> > >> aggregate)
> > >> >>>>>>>>>>        > >>>>      {
> > >> >>>>>>>>>>        > >>>>          Map<U, V> currentStateAsMap =
> > >> >>> asMap(aggregate);
> > >> >>>>>>>>>> <<
> > >> >>>>>>>>>>        imaginary
> > >> >>>>>>>>>>        > >>>>          U toModifyKey = mapper.apply(value);
> > >> >>>>>>>>>>        > >>>>              << this is the place where
> people
> > >> >>> actually
> > >> >>>>>>>>>>        gonna have
> > >> >>>>>>>>>>        > issues
> > >> >>>>>>>>>>        > >>>> and why you probably couldn't do it. we would
> > >> need
> > >> >>> to find
> > >> >>>>>>>>>>        a solution
> > >> >>>>>>>>>>        > here.
> > >> >>>>>>>>>>        > >>>> I didn't realize that yet.
> > >> >>>>>>>>>>        > >>>>              << we propagate the field in the
> > >> >>> joiner, so
> > >> >>>>>>>>>>        that we can
> > >> >>>>>>>>>>        > pick
> > >> >>>>>>>>>>        > >>>> it up in an aggregate. Probably you have not
> > >> thought
> > >> >>> of
> > >> >>>>>>>>>>        this in your
> > >> >>>>>>>>>>        > >>>> approach right?
> > >> >>>>>>>>>>        > >>>>              << I am very open to find a
> > generic
> > >> >>> solution
> > >> >>>>>>>>>>        here. In my
> > >> >>>>>>>>>>        > >>>> honest opinion this is broken in
> > >> KTableImpl.GroupBy
> > >> >>> that
> > >> >>>>>>>>>> it
> > >> >>>>>>>>>>        looses
> > >> >>>>>>>>>>        > the keys
> > >> >>>>>>>>>>        > >>>> and only maintains the aggregate key.
> > >> >>>>>>>>>>        > >>>>              << I abstracted it away back
> then
> > >> way
> > >> >>> before
> > >> >>>>>>>>>> i
> > >> >>>>>>>>>> was
> > >> >>>>>>>>>>        > thinking
> > >> >>>>>>>>>>        > >>>> of oneToMany join. That is why I didn't
> realize
> > >> its
> > >> >>>>>>>>>>        significance here.
> > >> >>>>>>>>>>        > >>>>              << Opinions?
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>          for (V m : current)
> > >> >>>>>>>>>>        > >>>>          {
> > >> >>>>>>>>>>        > >>>> currentStateAsMap.put(mapper.apply(m), m);
> > >> >>>>>>>>>>        > >>>>          }
> > >> >>>>>>>>>>        > >>>>          if (isAdder)
> > >> >>>>>>>>>>        > >>>>          {
> > >> >>>>>>>>>>        > >>>> currentStateAsMap.put(toModifyKey, value);
> > >> >>>>>>>>>>        > >>>>          }
> > >> >>>>>>>>>>        > >>>>          else
> > >> >>>>>>>>>>        > >>>>          {
> > >> >>>>>>>>>>        > >>>> currentStateAsMap.remove(toModifyKey);
> > >> >>>>>>>>>>        > >>>> if(currentStateAsMap.isEmpty()){
> > >> >>>>>>>>>>        > >>>>                  return null;
> > >> >>>>>>>>>>        > >>>>              }
> > >> >>>>>>>>>>        > >>>>          }
> > >> >>>>>>>>>>        > >>>>          retrun
> > >> asAggregateType(currentStateAsMap)
> > >> >>>>>>>>>>        > >>>>      }
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>>
> > >> >>>>>>>>>>        > >>>> Thanks,
> > >> >>>>>>>>>>        > >>>>> Adam
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
> Filipiak <
> > >> >>>>>>>>>>        > Jan.Filipiak@trivago.com <mailto:
> > >> Jan.Filipiak@trivago.com
> > >> >>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > >>>>> wrote:
> > >> >>>>>>>>>>        > >>>>>
> > >> >>>>>>>>>>        > >>>>> Thanks Adam for bringing Matthias to speed!
> > >> >>>>>>>>>>        > >>>>>> about the differences. I think re-keying
> back
> > >> >>> should be
> > >> >>>>>>>>>>        optional at
> > >> >>>>>>>>>>        > >>>>>> best.
> > >> >>>>>>>>>>        > >>>>>> I would say we return a KScatteredTable
> with
> > >> >>> reshuffle()
> > >> >>>>>>>>>>        returning
> > >> >>>>>>>>>>        > >>>>>> KTable<originalKey,Joined> to make the
> > >> backwards
> > >> >>>>>>>>>>        repartitioning
> > >> >>>>>>>>>>        > >>>>>> optional.
> > >> >>>>>>>>>>        > >>>>>> I am also in a big favour of doing the out
> of
> > >> order
> > >> >>>>>>>>>>        processing using
> > >> >>>>>>>>>>        > >>>>>> group
> > >> >>>>>>>>>>        > >>>>>> by instead high water mark tracking.
> > >> >>>>>>>>>>        > >>>>>> Just because unbounded growth is just
> scary +
> > >> It
> > >> >>> saves
> > >> >>>>>>>>>> us
> > >> >>>>>>>>>>        the header
> > >> >>>>>>>>>>        > >>>>>> stuff.
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>> I think the abstraction of always
> > >> repartitioning
> > >> >>> back is
> > >> >>>>>>>>>>        just not so
> > >> >>>>>>>>>>        > >>>>>> strong. Like the work has been done before
> we
> > >> >>> partition
> > >> >>>>>>>>>>        back and
> > >> >>>>>>>>>>        > >>>>>> grouping
> > >> >>>>>>>>>>        > >>>>>> by something else afterwards is really
> > common.
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> > >> >>>>>>>>>>        > >>>>>>
> > >> >>>>>>>>>>        > >>>>>> Hi Matthias
> > >> >>>>>>>>>>        > >>>>>>> Thank you for your feedback, I do
> appreciate
> > >> it!
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> While name spacing would be possible, it
> > would
> > >> >>> require
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>        > deserialize
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
> > >> overhead. I
> > >> >>> would
> > >> >>>>>>>>>>        suggest to
> > >> >>>>>>>>>>        > no
> > >> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead.
> If
> > >> this
> > >> >>>>>>>>>>
> > >> >>>>>>>>> becomes a
> > >> >>>>>>>
> > >> >>>>>>>>        > problem in
> > >> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> > >> later
> > >> >>> on.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Agreed. I will go with using a reserved
> > >> string
> > >> >>> and
> > >> >>>>>>>>>>        document it.
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> My main concern about the design it the
> type
> > >> of
> > >> >>> the
> > >> >>>>>>>>>>        result KTable:
> > >> >>>>>>>>>>        > If
> > >> >>>>>>>>>>        > >>>>>>> I
> > >> >>>>>>>>>>        > >>>>>>> understood the proposal correctly,
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> In your example, you have table1 and
> table2
> > >> >>> swapped.
> > >> >>>>>>>>>>        Here is how it
> > >> >>>>>>>>>>        > >>>>>>> works
> > >> >>>>>>>>>>        > >>>>>>> currently:
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> 1) table1 has the records that contain the
> > >> >>> foreign key
> > >> >>>>>>>>>>        within their
> > >> >>>>>>>>>>        > >>>>>>> value.
> > >> >>>>>>>>>>        > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> > >> >>>>>>>>>> <b,(fk=A,bar=2)>,
> > >> >>>>>>>>>>        > >>>>>>> <c,(fk=B,bar=3)>
> > >> >>>>>>>>>>        > >>>>>>> table2 input stream: <A,X>, <B,Y>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> 2) A Value mapper is required to extract
> the
> > >> >>> foreign
> > >> >>>>>>>>>> key.
> > >> >>>>>>>>>>        > >>>>>>> table1 foreign key mapper: ( value =>
> > >> value.fk
> > >> >>>>>>>>>>        <http://value.fk> )
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> The mapper is applied to each element in
> > >> table1,
> > >> >>> and a
> > >> >>>>>>>>>>        new combined
> > >> >>>>>>>>>>        > >>>>>>> key is
> > >> >>>>>>>>>>        > >>>>>>> made:
> > >> >>>>>>>>>>        > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> > >> >>>>>>>>>> (fk=A,bar=2)>,
> > >> >>>>>>>>>>        <B-c,
> > >> >>>>>>>>>>        > >>>>>>> (fk=B,bar=3)>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> 3) The rekeyed events are copartitioned
> with
> > >> >>> table2:
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> a) Stream Thread with Partition 0:
> > >> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
> > >> <A-b,
> > >> >>>>>>>>>>        (fk=A,bar=2)>
> > >> >>>>>>>>>>        > >>>>>>> Table2: <A,X>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> b) Stream Thread with Partition 1:
> > >> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> > >> >>>>>>>>>>        > >>>>>>> Table2: <B,Y>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> 4) From here, they can be joined together
> > >> locally
> > >> >>> by
> > >> >>>>>>>>>>        applying the
> > >> >>>>>>>>>>        > >>>>>>> joiner
> > >> >>>>>>>>>>        > >>>>>>> function.
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> At this point, Jan's design and my design
> > >> >>> deviate. My
> > >> >>>>>>>>>>        design goes
> > >> >>>>>>>>>>        > on
> > >> >>>>>>>>>>        > >>>>>>> to
> > >> >>>>>>>>>>        > >>>>>>> repartition the data post-join and resolve
> > >> >>> out-of-order
> > >> >>>>>>>>>>        arrival of
> > >> >>>>>>>>>>        > >>>>>>> records,
> > >> >>>>>>>>>>        > >>>>>>> finally returning the data keyed just the
> > >> >>> original key.
> > >> >>>>>>>>>>        I do not
> > >> >>>>>>>>>>        > >>>>>>> expose
> > >> >>>>>>>>>>        > >>>>>>> the
> > >> >>>>>>>>>>        > >>>>>>> CombinedKey or any of the internals
> outside
> > >> of the
> > >> >>>>>>>>>>        joinOnForeignKey
> > >> >>>>>>>>>>        > >>>>>>> function. This does make for larger
> > footprint,
> > >> >>> but it
> > >> >>>>>>>>>>        removes all
> > >> >>>>>>>>>>        > >>>>>>> agency
> > >> >>>>>>>>>>        > >>>>>>> for resolving out-of-order arrivals and
> > >> handling
> > >> >>>>>>>>>>        CombinedKeys from
> > >> >>>>>>>>>>        > the
> > >> >>>>>>>>>>        > >>>>>>> user. I believe that this makes the
> function
> > >> much
> > >> >>>>>>>>>> easier
> > >> >>>>>>>>>>        to use.
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> Let me know if this helps resolve your
> > >> questions,
> > >> >>> and
> > >> >>>>>>>>>>        please feel
> > >> >>>>>>>>>>        > >>>>>>> free to
> > >> >>>>>>>>>>        > >>>>>>> add anything else on your mind.
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> Thanks again,
> > >> >>>>>>>>>>        > >>>>>>> Adam
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias
> J.
> > >> Sax <
> > >> >>>>>>>>>>        > >>>>>>> matthias@confluent.io <mailto:
> > >> >>> matthias@confluent.io>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> wrote:
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>> Hi,
> > >> >>>>>>>>>>        > >>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> I am just catching up on this thread. I
> did
> > >> not
> > >> >>> read
> > >> >>>>>>>>>>        everything so
> > >> >>>>>>>>>>        > >>>>>>>> far,
> > >> >>>>>>>>>>        > >>>>>>>> but want to share couple of initial
> > thoughts:
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Headers: I think there is a fundamental
> > >> >>> difference
> > >> >>>>>>>>>>        between header
> > >> >>>>>>>>>>        > >>>>>>>> usage
> > >> >>>>>>>>>>        > >>>>>>>> in this KIP and KP-258. For 258, we add
> > >> headers
> > >> >>> to
> > >> >>>>>>>>>>        changelog topic
> > >> >>>>>>>>>>        > >>>>>>>> that
> > >> >>>>>>>>>>        > >>>>>>>> are owned by Kafka Streams and nobody
> else
> > is
> > >> >>> supposed
> > >> >>>>>>>>>>        to write
> > >> >>>>>>>>>>        > into
> > >> >>>>>>>>>>        > >>>>>>>> them. In fact, no user header are written
> > >> into
> > >> >>> the
> > >> >>>>>>>>>>        changelog topic
> > >> >>>>>>>>>>        > >>>>>>>> and
> > >> >>>>>>>>>>        > >>>>>>>> thus, there are not conflicts.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Nevertheless, I don't see a big issue
> with
> > >> using
> > >> >>>>>>>>>>        headers within
> > >> >>>>>>>>>>        > >>>>>>>> Streams.
> > >> >>>>>>>>>>        > >>>>>>>> As long as we document it, we can have
> some
> > >> >>> "reserved"
> > >> >>>>>>>>>>        header keys
> > >> >>>>>>>>>>        > >>>>>>>> and
> > >> >>>>>>>>>>        > >>>>>>>> users are not allowed to use when
> > processing
> > >> >>> data with
> > >> >>>>>>>>>>        Kafka
> > >> >>>>>>>>>>        > Streams.
> > >> >>>>>>>>>>        > >>>>>>>> IMHO, this should be ok.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> I think there is a safe way to avoid
> > >> conflicts,
> > >> >>> since
> > >> >>>>>>>>>> these
> > >> >>>>>>>>>>        > headers
> > >> >>>>>>>>>>        > >>>>>>>> are
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>> only needed in internal topics (I
> think):
> > >> >>>>>>>>>>        > >>>>>>>>> For internal and changelog topics, we
> can
> > >> >>> namespace
> > >> >>>>>>>>>>        all headers:
> > >> >>>>>>>>>>        > >>>>>>>>> * user-defined headers are namespaced as
> > >> >>> "external."
> > >> >>>>>>>>>> +
> > >> >>>>>>>>>>        headerKey
> > >> >>>>>>>>>>        > >>>>>>>>> * internal headers are namespaced as
> > >> >>> "internal." +
> > >> >>>>>>>>>>        headerKey
> > >> >>>>>>>>>>        > >>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>> While name spacing would be possible, it
> > >> would
> > >> >>>>>>>>>> require
> > >> >>>>>>>>>>
> > >> >>>>>>>>> to
> > >> >>>>>>>
> > >> >>>>>>>>        > >>>>>>>> deserialize
> > >> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
> > >> overhead. I
> > >> >>> would
> > >> >>>>>>>>>>        suggest to
> > >> >>>>>>>>>>        > no
> > >> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead.
> If
> > >> this
> > >> >>>>>>>>>>
> > >> >>>>>>>>> becomes a
> > >> >>>>>>>
> > >> >>>>>>>>        > problem in
> > >> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> > >> later
> > >> >>> on.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> My main concern about the design it the
> > type
> > >> of
> > >> >>> the
> > >> >>>>>>>>>>        result KTable:
> > >> >>>>>>>>>>        > >>>>>>>> If I
> > >> >>>>>>>>>>        > >>>>>>>> understood the proposal correctly,
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> KTable<K1,V1> table1 = ...
> > >> >>>>>>>>>>        > >>>>>>>> KTable<K2,V2> table2 = ...
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> KTable<K1,V3> joinedTable =
> > >> >>> table1.join(table2,...);
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> implies that the `joinedTable` has the
> same
> > >> key
> > >> >>> as the
> > >> >>>>>>>>>>        left input
> > >> >>>>>>>>>>        > >>>>>>>> table.
> > >> >>>>>>>>>>        > >>>>>>>> IMHO, this does not work because if
> table2
> > >> >>> contains
> > >> >>>>>>>>>>        multiple rows
> > >> >>>>>>>>>>        > >>>>>>>> that
> > >> >>>>>>>>>>        > >>>>>>>> join with a record in table1 (what is the
> > >> main
> > >> >>> purpose
> > >> >>>>>>>>>>
> > >> >>>>>>>>> of
> > >> >>>>>>>
> > >> >>>>>>>> a
> > >> >>>>>>>>>>        > foreign
> > >> >>>>>>>>>>        > >>>>>>>> key
> > >> >>>>>>>>>>        > >>>>>>>> join), the result table would only
> contain
> > a
> > >> >>> single
> > >> >>>>>>>>>>        join result,
> > >> >>>>>>>>>>        > but
> > >> >>>>>>>>>>        > >>>>>>>> not
> > >> >>>>>>>>>>        > >>>>>>>> multiple.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Example:
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> table1 input stream: <A,X>
> > >> >>>>>>>>>>        > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> We use table2 value a foreign key to
> table1
> > >> key
> > >> >>> (ie,
> > >> >>>>>>>>>>        "A" joins).
> > >> >>>>>>>>>>        > If
> > >> >>>>>>>>>>        > >>>>>>>> the
> > >> >>>>>>>>>>        > >>>>>>>> result key is the same key as key of
> > table1,
> > >> this
> > >> >>>>>>>>>>        implies that the
> > >> >>>>>>>>>>        > >>>>>>>> result can either be <A, join(X,1)> or
> <A,
> > >> >>> join(X,2)>
> > >> >>>>>>>>>>        but not
> > >> >>>>>>>>>>        > both.
> > >> >>>>>>>>>>        > >>>>>>>> Because the share the same key, whatever
> > >> result
> > >> >>> record
> > >> >>>>>>>>>>        we emit
> > >> >>>>>>>>>>        > later,
> > >> >>>>>>>>>>        > >>>>>>>> overwrite the previous result.
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> This is the reason why Jan originally
> > >> proposed
> > >> >>> to use
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>        > combination
> > >> >>>>>>>>>>        > >>>>>>>> of
> > >> >>>>>>>>>>        > >>>>>>>> both primary keys of the input tables as
> > key
> > >> of
> > >> >>> the
> > >> >>>>>>>>>>        output table.
> > >> >>>>>>>>>>        > >>>>>>>> This
> > >> >>>>>>>>>>        > >>>>>>>> makes the keys of the output table unique
> > >> and we
> > >> >>> can
> > >> >>>>>>>>>>        store both in
> > >> >>>>>>>>>>        > >>>>>>>> the
> > >> >>>>>>>>>>        > >>>>>>>> output table:
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
> > >> >>> join(X,2)>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Thoughts?
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> -Matthias
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> > >> >>>>>>>>>>        > >>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>> Just on remark here.
> > >> >>>>>>>>>>        > >>>>>>>>> The high-watermark could be disregarded.
> > The
> > >> >>> decision
> > >> >>>>>>>>>>        about the
> > >> >>>>>>>>>>        > >>>>>>>>> forward
> > >> >>>>>>>>>>        > >>>>>>>>> depends on the size of the aggregated
> map.
> > >> >>>>>>>>>>        > >>>>>>>>> Only 1 element long maps would be
> unpacked
> > >> and
> > >> >>>>>>>>>>        forwarded. 0
> > >> >>>>>>>>>>        > element
> > >> >>>>>>>>>>        > >>>>>>>>> maps
> > >> >>>>>>>>>>        > >>>>>>>>> would be published as delete. Any other
> > >> count
> > >> >>>>>>>>>>        > >>>>>>>>> of map entries is in "waiting for
> correct
> > >> >>> deletes to
> > >> >>>>>>>>>>        > arrive"-state.
> > >> >>>>>>>>>>        > >>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
> wrote:
> > >> >>>>>>>>>>        > >>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>> It does look like I could replace the
> > second
> > >> >>>>>>>>>>        repartition store
> > >> >>>>>>>>>>        > and
> > >> >>>>>>>>>>        > >>>>>>>>>> highwater store with a groupBy and
> > reduce.
> > >> >>> However,
> > >> >>>>>>>>>>        it looks
> > >> >>>>>>>>>>        > like
> > >> >>>>>>>>>>        > >>>>>>>>>> I
> > >> >>>>>>>>>>        > >>>>>>>>>> would
> > >> >>>>>>>>>>        > >>>>>>>>>> still need to store the highwater value
> > >> within
> > >> >>> the
> > >> >>>>>>>>>>        materialized
> > >> >>>>>>>>>>        > >>>>>>>>>> store,
> > >> >>>>>>>>>>        > >>>>>>>>>>
> > >> >>>>>>>>>>        > >>>>>>>>>> to
> > >> >>>>>>>>>>        > >>>>>>>>> compare the arrival of out-of-order
> > records
> > >> >>> (assuming
> > >> >>>>>>>>>>
> > >> >>>>>>>>> my
> > >> >>>>>>>
> > >> >>>>>>>>        > >>>>>>>>> understanding
> > >> >>>>>>>>>>        > >>>>>>>>> of
> > >> >>>>>>>>>>        > >>>>>>>>> THIS is correct...). This in effect is
> the
> > >> same
> > >> >>> as
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>        design I
> > >> >>>>>>>>>>        > have
> > >> >>>>>>>>>>        > >>>>>>>>> now,
> > >> >>>>>>>>>>        > >>>>>>>>> just with the two tables merged
> together.
> > >> >>>>>>>>>>        > >>>>>>>>>
> > >> >>>>>>>>>>        >
> > >> >>>>>>>>>>        >
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> --
> > >> >>>>>>>> -- Guozhang
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> --
> > >> >>>>>>> -- Guozhang
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >> >
> > >>
> > >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi John

Thanks for the feedback. It is extremely valuable. Responses as follows:

1. I could definitely override the join function.

The API you proposed is fine by me:
KTable<K, VR> join(KTable<KO, VO> other,
                   ValueMapper<V, KO> foreignKeyExtractor,
                   ValueJoiner<V, VO, VR> joiner,
                   ManyToOneJoined manyToOneJoined, // join config object
                  );

2. Currently this is only an inner-join. The join logic is only executed
when records exist in both KTables. It could be expanded to do other joins
too, but I wanted to limit the scope in the first pass and focus on a
minimum product. If this is accepted, I could generate a ticket to work on
the outer and left join.

3a. True. I will revisit the input requirements of the function.

3b. It is only necessary if the user has custom-partitioned this or other
KTable. The current implementation assumes that it's the default hash if
nothing else is specified. However, in line with the other joins, it would
make sense to assume they are identically partitioned. I can remove it no
problem.

4. No longer relevant. See 6.

5. No longer relevant. See 6.

6. Brilliant! Thank you for this piece of feedback. This is me missing the
forest for the trees. I failed to remember that the events would indeed end
up back at their original node, and could be compared in such a way. I have
implemented this solution this morning and it seems to work. I will mull it
over a bit more to see if I can find any problematic race conditions.
Provided this is indeed sound, I believe you neatly sliced this problem
down to a much neater and simpler size.

7. We have been using this in production. The scale of data is generally
around the many being in the 100-500 entity range. Sometimes we do have
10000+ in the "many" and indeed it does take a bit longer. I suspect it
would be best to measure it directly, something like:

- 1 to 100 through 1 Million in a factor of 10
- Reduce RocksDB in-memory allocation to a minimal amount to exercise disk
seeks.

I am not sure how far down the path to go with regards to testing RocksDB
performance vs. Kafka Streams functionality, but I will take a look and get
back to you on this.

With regards to changing the structure to an unbounded value size, in my
use-cases we have already run into the size problem and I would prefer to
avoid a paging solution. I will focus on finding a means to performance
test before making any more design changes.

Thanks again John, I am very appreciative. I suspect this may need to move
back to DISCUSS though (which I am fine with)

Adam






On Thu, Nov 29, 2018 at 4:18 PM John Roesler <jo...@confluent.io> wrote:

> Actually, I've been thinking more about my feedback #1 on the method name,
> and I'm not so sure it's a good idea.
>
> Following SQL, the existing join methods are named according to handedness
> (inner/outer/left). All these joins are the same cardinality (1:1 joins). I
> think it would be a mistake to switch up the naming scheme and introduce a
> new method named for cardinality, like "manyToOneJoin".
>
> Can we actually just add a new overload to the existing joins? The javadoc
> could explain that it's a many-to-one join, and it would be differentiated
> by the presence of the "keyExtractor".
>
> Just to be safe, if we rename the proposed "joinOnForeignKey" to just
> "join", then we could rename "keyExtractor" to "foreignKeyExtractor" for
> clarity.
>
> Just to be unambiguous, including the other API feedback I gave, I'm
> proposing something like this:
>
> KTable<K, VR> join(KTable<KO, VO> other,
>                    ValueMapper<V, KO> foreignKeyExtractor,
>                    ValueJoiner<V, VO, VR> joiner,
>                    ManyToOneJoined manyToOneJoined, // join config object
>                   );
>
> The ManyToOneJoined config object would allow setting all 4 serdes and
> configuring materialization.
>
> Thanks for your consideration,
> -John
>
> On Thu, Nov 29, 2018 at 8:14 AM John Roesler <jo...@confluent.io> wrote:
>
> > Hi all,
> >
> > Sorry that this discussion petered out... I think the 2.1 release caused
> > an extended distraction that pushed it off everyone's radar (which was
> > precisely Adam's concern). Personally, I've also had some extend
> > distractions of my own that kept (and continue to keep) me preoccupied.
> >
> > However, calling for a vote did wake me up, so I guess Jan was on the
> > right track!
> >
> > I've gone back and reviewed the whole KIP document and the prior
> > discussion, and I'd like to offer a few thoughts:
> >
> > API Thoughts:
> >
> > 1. If I read the KIP right, you are proposing a many-to-one join. Could
> we
> > consider naming it manyToOneJoin? Or, if you prefer, flip the design
> around
> > and make it a oneToManyJoin?
> >
> > The proposed name "joinOnForeignKey" disguises the join type, and it
> seems
> > like it might trick some people into using it for a one-to-one join. This
> > would work, of course, but it would be super inefficient compared to a
> > simple rekey-and-join.
> >
> > 2. I might have missed it, but I don't think it's specified whether it's
> > an inner, outer, or left join. I'm guessing an outer join, as (neglecting
> > IQ), the rest can be achieved by filtering or by handling it in the
> > ValueJoiner.
> >
> > 3. The arg list to joinOnForeignKey doesn't look quite right.
> > 3a. Regarding Serialized: There are a few different paradigms in play in
> > the Streams API, so it's confusing, but instead of three Serialized
> args, I
> > think it would be better to have one that allows (optionally) setting
> the 4
> > incoming serdes. The result serde is defined by the Materialized. The
> > incoming serdes can be optional because they might already be available
> on
> > the source KTables, or the default serdes from the config might be
> > applicable.
> >
> > 3b. Is the StreamPartitioner necessary? The other joins don't allow
> > setting one, and it seems like it might actually be harmful, since the
> > rekey operation needs to produce results that are co-partitioned with the
> > "other" KTable.
> >
> > 4. I'm fine with the "reserved word" header, but I didn't actually follow
> > what Matthias meant about namespacing requiring "deserializing" the
> record
> > header. The headers are already Strings, so I don't think that
> > deserialization is required. If we applied the namespace at source nodes
> > and stripped it at sink nodes, this would be practically no overhead. The
> > advantage of the namespace idea is that no public API change wrt headers
> > needs to happen, and no restrictions need to be placed on users' headers.
> >
> > (Although I'm wondering if we can get away without the header at all...
> > stay tuned)
> >
> > 5. I also didn't follow the discussion about the HWM table growing
> without
> > bound. As I read it, the HWM table is effectively implementing OCC to
> > resolve the problem you noted with disordering when the rekey is
> > reversed... particularly notable when the FK changes. As such, it only
> > needs to track the most recent "version" (the offset in the source
> > partition) of each key. Therefore, it should have the same number of keys
> > as the source table at all times.
> >
> > I see that you are aware of KIP-258, which I think might be relevant in a
> > couple of ways. One: it's just about storing the timestamp in the state
> > store, but the ultimate idea is to effectively use the timestamp as an
> OCC
> > "version" to drop disordered updates. You wouldn't want to use the
> > timestamp for this operation, but if you were to use a similar mechanism
> to
> > store the source offset in the store alongside the re-keyed values, then
> > you could avoid a separate table.
> >
> > 6. You and Jan have been thinking about this for a long time, so I've
> > probably missed something here, but I'm wondering if we can avoid the HWM
> > tracking at all and resolve out-of-order during a final join instead...
> >
> > Let's say we're joining a left table (Integer K: Letter FK, (other data))
> > to a right table (Letter K: (some data)).
> >
> > Left table:
> > 1: (A, xyz)
> > 2: (B, asd)
> >
> > Right table:
> > A: EntityA
> > B: EntityB
> >
> > We could do a rekey as you proposed with a combined key, but not
> > propagating the value at all..
> > Rekey table:
> > A-1: (dummy value)
> > B-2: (dummy value)
> >
> > Which we then join with the right table to produce:
> > A-1: EntityA
> > B-2: EntityB
> >
> > Which gets rekeyed back:
> > 1: A, EntityA
> > 2: B, EntityB
> >
> > And finally we do the actual join:
> > Result table:
> > 1: ((A, xyz), EntityA)
> > 2: ((B, asd), EntityB)
> >
> > The thing is that in that last join, we have the opportunity to compare
> > the current FK in the left table with the incoming PK of the right table.
> > If they don't match, we just drop the event, since it must be outdated.
> >
> > In your KIP, you gave an example in which (1: A, xyz) gets updated to (1:
> > B, xyz), ultimately yielding a conundrum about whether the final state
> > should be (1: null) or (1: joined-on-B). With the algorithm above, you
> > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> > EntityB)). It seems like this does give you enough information to make
> the
> > right choice, regardless of disordering.
> >
> >
> > 7. Last thought... I'm a little concerned about the performance of the
> > range scans when records change in the right table. You've said that
> you've
> > been using the algorithm you presented in production for a while. Can you
> > give us a sense of the performance characteristics you've observed?
> >
> > I could only think of one alternative, but I'm not sure if it's better or
> > worse... If the first re-key only needs to preserve the original key, as
> I
> > proposed in #6, then we could store a vector of keys in the value:
> >
> > Left table:
> > 1: A,...
> > 2: B,...
> > 3: A,...
> >
> > Gets re-keyed:
> > A: [1, 3]
> > B: [2]
> >
> > Then, the rhs part of the join would only need a regular single-key
> > lookup. Of course we have to deal with the problem of large values, as
> > there's no bound on the number of lhs records that can reference rhs
> > records. Offhand, I'd say we could page the values, so when one row is
> past
> > the threshold, we append the key for the next page. Then in most cases,
> it
> > would be a single key lookup, but for large fan-out updates, it would be
> > one per (max value size)/(avg lhs key size).
> >
> > This seems more complex, though... Plus, I think there's some extra
> > tracking we'd need to do to know when to emit a retraction. For example,
> > when record 1 is deleted, the re-key table would just have (A: [3]). Some
> > kind of tombstone is needed so that the join result for 1 can also be
> > retracted.
> >
> > That's all!
> >
> > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> > discussion has been slow.
> > -John
> >
> >
> > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >> Id say you can just call the vote.
> >>
> >> that happens all the time, and if something comes up, it just goes back
> >> to discuss.
> >>
> >> would not expect to much attention with another another email in this
> >> thread.
> >>
> >> best Jan
> >>
> >> On 09.10.2018 13:56, Adam Bellemare wrote:
> >> > Hello Contributors
> >> >
> >> > I know that 2.1 is about to be released, but I do need to bump this to
> >> keep
> >> > visibility up. I am still intending to push this through once
> >> contributor
> >> > feedback is given.
> >> >
> >> > Main points that need addressing:
> >> > 1) Any way (or benefit) in structuring the current singular graph node
> >> into
> >> > multiple nodes? It has a whopping 25 parameters right now. I am a bit
> >> fuzzy
> >> > on how the optimizations are supposed to work, so I would appreciate
> any
> >> > help on this aspect.
> >> >
> >> > 2) Overall strategy for joining + resolving. This thread has much
> >> discourse
> >> > between Jan and I between the current highwater mark proposal and a
> >> groupBy
> >> > + reduce proposal. I am of the opinion that we need to strictly handle
> >> any
> >> > chance of out-of-order data and leave none of it up to the consumer.
> Any
> >> > comments or suggestions here would also help.
> >> >
> >> > 3) Anything else that you see that would prevent this from moving to a
> >> vote?
> >> >
> >> > Thanks
> >> >
> >> > Adam
> >> >
> >> >
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >> > wrote:
> >> >
> >> >> Hi Jan
> >> >>
> >> >> With the Stores.windowStoreBuilder and Stores.persistentWindowStore,
> >> you
> >> >> actually only need to specify the amount of segments you want and how
> >> large
> >> >> they are. To the best of my understanding, what happens is that the
> >> >> segments are automatically rolled over as new data with new
> timestamps
> >> are
> >> >> created. We use this exact functionality in some of the work done
> >> >> internally at my company. For reference, this is the hopping windowed
> >> store.
> >> >>
> >> >>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >> >>
> >> >> In the code that I have provided, there are going to be two 24h
> >> segments.
> >> >> When a record is put into the windowStore, it will be inserted at
> time
> >> T in
> >> >> both segments. The two segments will always overlap by 12h. As time
> >> goes on
> >> >> and new records are added (say at time T+12h+), the oldest segment
> >> will be
> >> >> automatically deleted and a new segment created. The records are by
> >> default
> >> >> inserted with the context.timestamp(), such that it is the record
> >> time, not
> >> >> the clock time, which is used.
> >> >>
> >> >> To the best of my understanding, the timestamps are retained when
> >> >> restoring from the changelog.
> >> >>
> >> >> Basically, this is heavy-handed way to deal with TTL at a
> >> segment-level,
> >> >> instead of at an individual record level.
> >> >>
> >> >> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> Jan.Filipiak@trivago.com
> >> >
> >> >> wrote:
> >> >>
> >> >>> Will that work? I expected it to blow up with ClassCastException or
> >> >>> similar.
> >> >>>
> >> >>> You either would have to specify the window you fetch/put or iterate
> >> >>> across all windows the key was found in right?
> >> >>>
> >> >>> I just hope the window-store doesn't check stream-time under the
> hoods
> >> >>> that would be a questionable interface.
> >> >>>
> >> >>> If it does: did you see my comment on checking all the windows
> >> earlier?
> >> >>> that would be needed to actually give reasonable time gurantees.
> >> >>>
> >> >>> Best
> >> >>>
> >> >>>
> >> >>>
> >> >>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >> >>>> Hi Jan
> >> >>>>
> >> >>>> Check for  " highwaterMat " in the PR. I only changed the state
> >> store,
> >> >>> not
> >> >>>> the ProcessorSupplier.
> >> >>>>
> >> >>>> Thanks,
> >> >>>> Adam
> >> >>>>
> >> >>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >> Jan.Filipiak@trivago.com
> >> >>>>
> >> >>>> wrote:
> >> >>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >> >>>>>
> >> >>>>>> @Guozhang
> >> >>>>>>
> >> >>>>>> Thanks for the information. This is indeed something that will be
> >> >>>>>> extremely
> >> >>>>>> useful for this KIP.
> >> >>>>>>
> >> >>>>>> @Jan
> >> >>>>>> Thanks for your explanations. That being said, I will not be
> moving
> >> >>> ahead
> >> >>>>>> with an implementation using reshuffle/groupBy solution as you
> >> >>> propose.
> >> >>>>>> That being said, if you wish to implement it yourself off of my
> >> >>> current PR
> >> >>>>>> and submit it as a competitive alternative, I would be more than
> >> >>> happy to
> >> >>>>>> help vet that as an alternate solution. As it stands right now, I
> >> do
> >> >>> not
> >> >>>>>> really have more time to invest into alternatives without there
> >> being
> >> >>> a
> >> >>>>>> strong indication from the binding voters which they would
> prefer.
> >> >>>>>>
> >> >>>>>>
> >> >>>>> Hey, total no worries. I think I personally gave up on the streams
> >> DSL
> >> >>> for
> >> >>>>> some time already, otherwise I would have pulled this KIP through
> >> >>> already.
> >> >>>>> I am currently reimplementing my own DSL based on PAPI.
> >> >>>>>
> >> >>>>>
> >> >>>>>> I will look at finishing up my PR with the windowed state store
> in
> >> the
> >> >>>>>> next
> >> >>>>>> week or so, exercising it via tests, and then I will come back
> for
> >> >>> final
> >> >>>>>> discussions. In the meantime, I hope that any of the binding
> voters
> >> >>> could
> >> >>>>>> take a look at the KIP in the wiki. I have updated it according
> to
> >> the
> >> >>>>>> latest plan:
> >> >>>>>>
> >> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >> >>>>>> Support+non-key+joining+in+KTable
> >> >>>>>>
> >> >>>>>> I have also updated the KIP PR to use a windowed store. This
> could
> >> be
> >> >>>>>> replaced by the results of KIP-258 whenever they are completed.
> >> >>>>>> https://github.com/apache/kafka/pull/5527
> >> >>>>>>
> >> >>>>>> Thanks,
> >> >>>>>>
> >> >>>>>> Adam
> >> >>>>>>
> >> >>>>>
> >> >>>>> Is the HighWatermarkResolverProccessorsupplier already updated in
> >> the
> >> >>> PR?
> >> >>>>> expected it to change to Windowed<K>,Long Missing something?
> >> >>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as
> >> it
> >> >>> is
> >> >>>>>>> for
> >> >>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
> >> >>> want to
> >> >>>>>>> have "handling out-of-order data for source KTable" such that
> >> >>> instead of
> >> >>>>>>> blindly apply the updates to the materialized store, i.e.
> >> following
> >> >>>>>>> offset
> >> >>>>>>> ordering, we will reject updates that are older than the current
> >> >>> key's
> >> >>>>>>> timestamps, i.e. following timestamp ordering.
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> Guozhang
> >> >>>>>>>
> >> >>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >> >>>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>> Hello Adam,
> >> >>>>>>>>
> >> >>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
> >> high
> >> >>>>>>>> watermark store, now altered to be replaced with a window
> >> store), I
> >> >>>>>>>> think
> >> >>>>>>>> another current on-going KIP may actually help:
> >> >>>>>>>>
> >> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> This is for adding the timestamp into a key-value store (i.e.
> >> only
> >> >>> for
> >> >>>>>>>> non-windowed KTable), and then one of its usage, as described
> in
> >> >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we
> can
> >> >>> then
> >> >>>>>>>> "reject" updates from the source topics if its timestamp is
> >> smaller
> >> >>> than
> >> >>>>>>>> the current key's latest update timestamp. I think it is very
> >> >>> similar to
> >> >>>>>>>> what you have in mind for high watermark based filtering, while
> >> you
> >> >>> only
> >> >>>>>>>> need to make sure that the timestamps of the joining records
> are
> >> >>>>>>>>
> >> >>>>>>> correctly
> >> >>>>>>>
> >> >>>>>>>> inherited though the whole topology to the final stage.
> >> >>>>>>>>
> >> >>>>>>>> Note that this KIP is for key-value store and hence
> non-windowed
> >> >>> KTables
> >> >>>>>>>> only, but for windowed KTables we do not really have a good
> >> support
> >> >>> for
> >> >>>>>>>> their joins anyways (
> >> >>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >> >>>>>>>> I
> >> >>>>>>>> think we can just consider non-windowed KTable-KTable non-key
> >> joins
> >> >>> for
> >> >>>>>>>> now. In which case, KIP-258 should help.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> Guozhang
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >> >>> Jan.Filipiak@trivago.com
> >> >>>>>>>>>
> >> >>>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >> >>>>>>>>>
> >> >>>>>>>>> Hi Guozhang
> >> >>>>>>>>>>
> >> >>>>>>>>>> Current highwater mark implementation would grow endlessly
> >> based
> >> >>> on
> >> >>>>>>>>>> primary key of original event. It is a pair of (<this table
> >> >>> primary
> >> >>>>>>>>>>
> >> >>>>>>>>> key>,
> >> >>>>>>>
> >> >>>>>>>> <highest offset seen for that key>). This is used to
> >> differentiate
> >> >>>>>>>>>>
> >> >>>>>>>>> between
> >> >>>>>>>
> >> >>>>>>>> late arrivals and new updates. My newest proposal would be to
> >> >>> replace
> >> >>>>>>>>>>
> >> >>>>>>>>> it
> >> >>>>>>>
> >> >>>>>>>> with a Windowed state store of Duration N. This would allow the
> >> same
> >> >>>>>>>>>> behaviour, but cap the size based on time. This should allow
> >> for
> >> >>> all
> >> >>>>>>>>>> late-arriving events to be processed, and should be
> >> customizable
> >> >>> by
> >> >>>>>>>>>> the
> >> >>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
> minutes
> >> of
> >> >>>>>>>>>>
> >> >>>>>>>>> window,
> >> >>>>>>>
> >> >>>>>>>> or perhaps 7 days...).
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Adam, using time based retention can do the trick here.
> Even
> >> >>> if I
> >> >>>>>>>>> would still like to see the automatic repartitioning optional
> >> >>> since I
> >> >>>>>>>>>
> >> >>>>>>>> would
> >> >>>>>>>
> >> >>>>>>>> just reshuffle again. With windowed store I am a little bit
> >> >>> sceptical
> >> >>>>>>>>>
> >> >>>>>>>> about
> >> >>>>>>>
> >> >>>>>>>> how to determine the window. So esentially one could run into
> >> >>> problems
> >> >>>>>>>>>
> >> >>>>>>>> when
> >> >>>>>>>
> >> >>>>>>>> the rapid change happens near a window border. I will check you
> >> >>>>>>>>> implementation in detail, if its problematic, we could still
> >> check
> >> >>>>>>>>> _all_
> >> >>>>>>>>> windows on read with not to bad performance impact I guess.
> Will
> >> >>> let
> >> >>>>>>>>> you
> >> >>>>>>>>> know if the implementation would be correct as is. I wouldn't
> >> not
> >> >>> like
> >> >>>>>>>>>
> >> >>>>>>>> to
> >> >>>>>>>
> >> >>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> >> timestamp(B).
> >> >>> I
> >> >>>>>>>>>
> >> >>>>>>>> think
> >> >>>>>>>
> >> >>>>>>>> we can't expect that.
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> @Jan
> >> >>>>>>>>>> I believe I understand what you mean now - thanks for the
> >> >>> diagram, it
> >> >>>>>>>>>> did really help. You are correct that I do not have the
> >> original
> >> >>>>>>>>>>
> >> >>>>>>>>> primary
> >> >>>>>>>
> >> >>>>>>>> key available, and I can see that if it was available then you
> >> >>> would be
> >> >>>>>>>>>> able to add and remove events from the Map. That being said,
> I
> >> >>>>>>>>>>
> >> >>>>>>>>> encourage
> >> >>>>>>>
> >> >>>>>>>> you to finish your diagrams / charts just for clarity for
> >> everyone
> >> >>>>>>>>>>
> >> >>>>>>>>> else.
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> >> >>> understand
> >> >>>>>>>>>>
> >> >>>>>>>>> the benefits for the rest. Sorry about the original primary
> >> key, We
> >> >>>>>>>>> have
> >> >>>>>>>>> join and Group by implemented our own in PAPI and basically
> not
> >> >>> using
> >> >>>>>>>>>
> >> >>>>>>>> any
> >> >>>>>>>
> >> >>>>>>>> DSL (Just the abstraction). Completely missed that in original
> >> DSL
> >> >>> its
> >> >>>>>>>>>
> >> >>>>>>>> not
> >> >>>>>>>
> >> >>>>>>>> there and just assumed it. total brain mess up on my end. Will
> >> >>> finish
> >> >>>>>>>>>
> >> >>>>>>>> the
> >> >>>>>>>
> >> >>>>>>>> chart as soon as i get a quite evening this week.
> >> >>>>>>>>>
> >> >>>>>>>>> My follow up question for you is, won't the Map stay inside
> the
> >> >>> State
> >> >>>>>>>>>
> >> >>>>>>>>>> Store indefinitely after all of the changes have propagated?
> >> Isn't
> >> >>>>>>>>>> this
> >> >>>>>>>>>> effectively the same as a highwater mark state store?
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thing is that if the map is empty, substractor is gonna
> return
> >> >>> `null`
> >> >>>>>>>>>
> >> >>>>>>>> and
> >> >>>>>>>
> >> >>>>>>>> the key is removed from the keyspace. But there is going to be
> a
> >> >>> store
> >> >>>>>>>>> 100%, the good thing is that I can use this store directly for
> >> >>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
> >> >>> satisfying
> >> >>>>>>>>> all gurantees needed for further groupby / join. The Windowed
> >> >>> store is
> >> >>>>>>>>>
> >> >>>>>>>> not
> >> >>>>>>>
> >> >>>>>>>> keeping the values, so for the next statefull operation we
> would
> >> >>>>>>>>> need to instantiate an extra store. or we have the window
> store
> >> >>> also
> >> >>>>>>>>>
> >> >>>>>>>> have
> >> >>>>>>>
> >> >>>>>>>> the values then.
> >> >>>>>>>>>
> >> >>>>>>>>> Long story short. if we can flip in a custom group by before
> >> >>>>>>>>> repartitioning to the original primary key i think it would
> help
> >> >>> the
> >> >>>>>>>>>
> >> >>>>>>>> users
> >> >>>>>>>
> >> >>>>>>>> big time in building efficient apps. Given the original primary
> >> key
> >> >>>>>>>>>
> >> >>>>>>>> issue I
> >> >>>>>>>
> >> >>>>>>>> understand that we do not have a solid foundation to build on.
> >> >>>>>>>>> Leaving primary key carry along to the user. very
> unfortunate. I
> >> >>> could
> >> >>>>>>>>> understand the decision goes like that. I do not think its a
> >> good
> >> >>>>>>>>>
> >> >>>>>>>> decision.
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks
> >> >>>>>>>>>> Adam
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >> >>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >> dumbreprajakta311@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>>>        please remove me from this group
> >> >>>>>>>>>>
> >> >>>>>>>>>>        On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >> >>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>        wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > Hi Adam,
> >> >>>>>>>>>>        >
> >> >>>>>>>>>>        > give me some time, will make such a chart. last
> time i
> >> >>> didn't
> >> >>>>>>>>>>        get along
> >> >>>>>>>>>>        > well with giphy and ruined all your charts.
> >> >>>>>>>>>>        > Hopefully i can get it done today
> >> >>>>>>>>>>        >
> >> >>>>>>>>>>        > On 08.09.2018 16:00, Adam Bellemare wrote:
> >> >>>>>>>>>>        > > Hi Jan
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > > I have included a diagram of what I attempted on
> the
> >> >>> KIP.
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        >
> >> >>>>>>>>>>
> >> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >> >>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >> >>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >> >>>>>>>>>>        <
> >> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >> >>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >> >>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > > I attempted this back at the start of my own
> >> >>> implementation
> >> >>>>>>>>>> of
> >> >>>>>>>>>>        this
> >> >>>>>>>>>>        > > solution, and since I could not get it to work I
> >> have
> >> >>> since
> >> >>>>>>>>>>        discarded the
> >> >>>>>>>>>>        > > code. At this point in time, if you wish to
> continue
> >> >>> pursuing
> >> >>>>>>>>>>        for your
> >> >>>>>>>>>>        > > groupBy solution, I ask that you please create a
> >> >>> diagram on
> >> >>>>>>>>>>        the KIP
> >> >>>>>>>>>>        > > carefully explaining your solution. Please feel
> >> free to
> >> >>> use
> >> >>>>>>>>>>        the image I
> >> >>>>>>>>>>        > > just posted as a starting point. I am having
> trouble
> >> >>>>>>>>>>        understanding your
> >> >>>>>>>>>>        > > explanations but I think that a carefully
> >> constructed
> >> >>> diagram
> >> >>>>>>>>>>        will clear
> >> >>>>>>>>>>        > up
> >> >>>>>>>>>>        > > any misunderstandings. Alternately, please post a
> >> >>>>>>>>>>        comprehensive PR with
> >> >>>>>>>>>>        > > your solution. I can only guess at what you mean,
> >> and
> >> >>> since I
> >> >>>>>>>>>>        value my
> >> >>>>>>>>>>        > own
> >> >>>>>>>>>>        > > time as much as you value yours, I believe it is
> >> your
> >> >>>>>>>>>>        responsibility to
> >> >>>>>>>>>>        > > provide an implementation instead of me trying to
> >> guess.
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > > Adam
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >> >>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > > wrote:
> >> >>>>>>>>>>        > >
> >> >>>>>>>>>>        > >> Hi James,
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >> nice to see you beeing interested. kafka streams
> at
> >> >>> this
> >> >>>>>>>>>>        point supports
> >> >>>>>>>>>>        > >> all sorts of joins as long as both streams have
> the
> >> >>> same
> >> >>>>>>>>>> key.
> >> >>>>>>>>>>        > >> Adam is currently implementing a join where a
> >> KTable
> >> >>> and a
> >> >>>>>>>>>>        KTable can
> >> >>>>>>>>>>        > have
> >> >>>>>>>>>>        > >> a one to many relation ship (1:n). We exploit
> that
> >> >>> rocksdb
> >> >>>>>>>>>> is
> >> >>>>>>>>>>
> >> >>>>>>>>> a
> >> >>>>>>>
> >> >>>>>>>>        > >> datastore that keeps data sorted (At least exposes
> an
> >> >>> API to
> >> >>>>>>>>>>        access the
> >> >>>>>>>>>>        > >> stored data in a sorted fashion).
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >> I think the technical caveats are well understood
> >> now
> >> >>> and we
> >> >>>>>>>>>>
> >> >>>>>>>>> are
> >> >>>>>>>
> >> >>>>>>>>        > basically
> >> >>>>>>>>>>        > >> down to philosophy and API Design ( when Adam
> sees
> >> my
> >> >>> newest
> >> >>>>>>>>>>        message).
> >> >>>>>>>>>>        > >> I have a lengthy track record of loosing those
> >> kinda
> >> >>>>>>>>>>        arguments within
> >> >>>>>>>>>>        > the
> >> >>>>>>>>>>        > >> streams community and I have no clue why. So I
> >> >>> literally
> >> >>>>>>>>>>        can't wait for
> >> >>>>>>>>>>        > you
> >> >>>>>>>>>>        > >> to churn through this thread and give you opinion
> >> on
> >> >>> how we
> >> >>>>>>>>>>        should
> >> >>>>>>>>>>        > design
> >> >>>>>>>>>>        > >> the return type of the oneToManyJoin and how many
> >> >>> power we
> >> >>>>>>>>>>        want to give
> >> >>>>>>>>>>        > to
> >> >>>>>>>>>>        > >> the user vs "simplicity" (where simplicity isn't
> >> >>> really that
> >> >>>>>>>>>>        as users
> >> >>>>>>>>>>        > still
> >> >>>>>>>>>>        > >> need to understand it I argue)
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >> waiting for you to join in on the discussion
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >> Best Jan
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >> On 07.09.2018 15:49, James Kwan wrote:
> >> >>>>>>>>>>        > >>
> >> >>>>>>>>>>        > >>> I am new to this group and I found this subject
> >> >>>>>>>>>>        interesting.  Sounds
> >> >>>>>>>>>>        > like
> >> >>>>>>>>>>        > >>> you guys want to implement a join table of two
> >> >>> streams? Is
> >> >>>>>>>>>> there
> >> >>>>>>>>>>        > somewhere
> >> >>>>>>>>>>        > >>> I can see the original requirement or proposal?
> >> >>>>>>>>>>        > >>>
> >> >>>>>>>>>>        > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >> >>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > >>>> wrote:
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>> I'm currently testing using a Windowed Store
> to
> >> >>> store the
> >> >>>>>>>>>>        highwater
> >> >>>>>>>>>>        > >>>>> mark.
> >> >>>>>>>>>>        > >>>>> By all indications this should work fine, with
> >> the
> >> >>> caveat
> >> >>>>>>>>>>        being that
> >> >>>>>>>>>>        > it
> >> >>>>>>>>>>        > >>>>> can
> >> >>>>>>>>>>        > >>>>> only resolve out-of-order arrival for up to
> the
> >> >>> size of
> >> >>>>>>>>>>        the window
> >> >>>>>>>>>>        > (ie:
> >> >>>>>>>>>>        > >>>>> 24h, 72h, etc). This would remove the
> >> possibility
> >> >>> of it
> >> >>>>>>>>>>
> >> >>>>>>>>> being
> >> >>>>>>>
> >> >>>>>>>>        > unbounded
> >> >>>>>>>>>>        > >>>>> in
> >> >>>>>>>>>>        > >>>>> size.
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>> With regards to Jan's suggestion, I believe
> >> this is
> >> >>> where
> >> >>>>>>>>>>        we will
> >> >>>>>>>>>>        > have
> >> >>>>>>>>>>        > >>>>> to
> >> >>>>>>>>>>        > >>>>> remain in disagreement. While I do not
> disagree
> >> >>> with your
> >> >>>>>>>>>>        statement
> >> >>>>>>>>>>        > >>>>> about
> >> >>>>>>>>>>        > >>>>> there likely to be additional joins done in a
> >> >>> real-world
> >> >>>>>>>>>>        workflow, I
> >> >>>>>>>>>>        > do
> >> >>>>>>>>>>        > >>>>> not
> >> >>>>>>>>>>        > >>>>> see how you can conclusively deal with
> >> out-of-order
> >> >>>>>>>>>> arrival
> >> >>>>>>>>>> of
> >> >>>>>>>>>>        > >>>>> foreign-key
> >> >>>>>>>>>>        > >>>>> changes and subsequent joins. I have attempted
> >> what
> >> >>> I
> >> >>>>>>>>>>        think you have
> >> >>>>>>>>>>        > >>>>> proposed (without a high-water, using groupBy
> >> and
> >> >>> reduce)
> >> >>>>>>>>>>        and found
> >> >>>>>>>>>>        > >>>>> that if
> >> >>>>>>>>>>        > >>>>> the foreign key changes too quickly, or the
> >> load on
> >> >>> a
> >> >>>>>>>>>>        stream thread
> >> >>>>>>>>>>        > is
> >> >>>>>>>>>>        > >>>>> too
> >> >>>>>>>>>>        > >>>>> high, the joined messages will arrive
> >> out-of-order
> >> >>> and be
> >> >>>>>>>>>>        incorrectly
> >> >>>>>>>>>>        > >>>>> propagated, such that an intermediate event is
> >> >>>>>>>>>> represented
> >> >>>>>>>>>>        as the
> >> >>>>>>>>>>        > final
> >> >>>>>>>>>>        > >>>>> event.
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>> Can you shed some light on your groupBy
> >> >>> implementation.
> >> >>>>>>>>>>        There must be
> >> >>>>>>>>>>        > >>>> some sort of flaw in it.
> >> >>>>>>>>>>        > >>>> I have a suspicion where it is, I would just
> >> like to
> >> >>>>>>>>>>        confirm. The idea
> >> >>>>>>>>>>        > >>>> is bullet proof and it must be
> >> >>>>>>>>>>        > >>>> an implementation mess up. I would like to
> >> clarify
> >> >>> before
> >> >>>>>>>>>>        we draw a
> >> >>>>>>>>>>        > >>>> conclusion.
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>    Repartitioning the scattered events back to
> >> their
> >> >>>>>>>>>>
> >> >>>>>>>>> original
> >> >>>>>>>
> >> >>>>>>>>        > >>>>> partitions is the only way I know how to
> >> conclusively
> >> >>> deal
> >> >>>>>>>>>>        with
> >> >>>>>>>>>>        > >>>>> out-of-order events in a given time frame, and
> >> to
> >> >>> ensure
> >> >>>>>>>>>>        that the
> >> >>>>>>>>>>        > data
> >> >>>>>>>>>>        > >>>>> is
> >> >>>>>>>>>>        > >>>>> eventually consistent with the input events.
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>> If you have some code to share that
> illustrates
> >> your
> >> >>>>>>>>>>        approach, I
> >> >>>>>>>>>>        > would
> >> >>>>>>>>>>        > >>>>> be
> >> >>>>>>>>>>        > >>>>> very grateful as it would remove any
> >> >>> misunderstandings
> >> >>>>>>>>>>        that I may
> >> >>>>>>>>>>        > have.
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>> ah okay you were looking for my code. I don't
> >> have
> >> >>>>>>>>>>        something easily
> >> >>>>>>>>>>        > >>>> readable here as its bloated with OO-patterns.
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>> its anyhow trivial:
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>> @Override
> >> >>>>>>>>>>        > >>>>      public T apply(K aggKey, V value, T
> >> aggregate)
> >> >>>>>>>>>>        > >>>>      {
> >> >>>>>>>>>>        > >>>>          Map<U, V> currentStateAsMap =
> >> >>> asMap(aggregate);
> >> >>>>>>>>>> <<
> >> >>>>>>>>>>        imaginary
> >> >>>>>>>>>>        > >>>>          U toModifyKey = mapper.apply(value);
> >> >>>>>>>>>>        > >>>>              << this is the place where people
> >> >>> actually
> >> >>>>>>>>>>        gonna have
> >> >>>>>>>>>>        > issues
> >> >>>>>>>>>>        > >>>> and why you probably couldn't do it. we would
> >> need
> >> >>> to find
> >> >>>>>>>>>>        a solution
> >> >>>>>>>>>>        > here.
> >> >>>>>>>>>>        > >>>> I didn't realize that yet.
> >> >>>>>>>>>>        > >>>>              << we propagate the field in the
> >> >>> joiner, so
> >> >>>>>>>>>>        that we can
> >> >>>>>>>>>>        > pick
> >> >>>>>>>>>>        > >>>> it up in an aggregate. Probably you have not
> >> thought
> >> >>> of
> >> >>>>>>>>>>        this in your
> >> >>>>>>>>>>        > >>>> approach right?
> >> >>>>>>>>>>        > >>>>              << I am very open to find a
> generic
> >> >>> solution
> >> >>>>>>>>>>        here. In my
> >> >>>>>>>>>>        > >>>> honest opinion this is broken in
> >> KTableImpl.GroupBy
> >> >>> that
> >> >>>>>>>>>> it
> >> >>>>>>>>>>        looses
> >> >>>>>>>>>>        > the keys
> >> >>>>>>>>>>        > >>>> and only maintains the aggregate key.
> >> >>>>>>>>>>        > >>>>              << I abstracted it away back then
> >> way
> >> >>> before
> >> >>>>>>>>>> i
> >> >>>>>>>>>> was
> >> >>>>>>>>>>        > thinking
> >> >>>>>>>>>>        > >>>> of oneToMany join. That is why I didn't realize
> >> its
> >> >>>>>>>>>>        significance here.
> >> >>>>>>>>>>        > >>>>              << Opinions?
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>          for (V m : current)
> >> >>>>>>>>>>        > >>>>          {
> >> >>>>>>>>>>        > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >> >>>>>>>>>>        > >>>>          }
> >> >>>>>>>>>>        > >>>>          if (isAdder)
> >> >>>>>>>>>>        > >>>>          {
> >> >>>>>>>>>>        > >>>> currentStateAsMap.put(toModifyKey, value);
> >> >>>>>>>>>>        > >>>>          }
> >> >>>>>>>>>>        > >>>>          else
> >> >>>>>>>>>>        > >>>>          {
> >> >>>>>>>>>>        > >>>> currentStateAsMap.remove(toModifyKey);
> >> >>>>>>>>>>        > >>>> if(currentStateAsMap.isEmpty()){
> >> >>>>>>>>>>        > >>>>                  return null;
> >> >>>>>>>>>>        > >>>>              }
> >> >>>>>>>>>>        > >>>>          }
> >> >>>>>>>>>>        > >>>>          retrun
> >> asAggregateType(currentStateAsMap)
> >> >>>>>>>>>>        > >>>>      }
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>>
> >> >>>>>>>>>>        > >>>> Thanks,
> >> >>>>>>>>>>        > >>>>> Adam
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> >> >>>>>>>>>>        > Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >> >>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > >>>>> wrote:
> >> >>>>>>>>>>        > >>>>>
> >> >>>>>>>>>>        > >>>>> Thanks Adam for bringing Matthias to speed!
> >> >>>>>>>>>>        > >>>>>> about the differences. I think re-keying back
> >> >>> should be
> >> >>>>>>>>>>        optional at
> >> >>>>>>>>>>        > >>>>>> best.
> >> >>>>>>>>>>        > >>>>>> I would say we return a KScatteredTable with
> >> >>> reshuffle()
> >> >>>>>>>>>>        returning
> >> >>>>>>>>>>        > >>>>>> KTable<originalKey,Joined> to make the
> >> backwards
> >> >>>>>>>>>>        repartitioning
> >> >>>>>>>>>>        > >>>>>> optional.
> >> >>>>>>>>>>        > >>>>>> I am also in a big favour of doing the out of
> >> order
> >> >>>>>>>>>>        processing using
> >> >>>>>>>>>>        > >>>>>> group
> >> >>>>>>>>>>        > >>>>>> by instead high water mark tracking.
> >> >>>>>>>>>>        > >>>>>> Just because unbounded growth is just scary +
> >> It
> >> >>> saves
> >> >>>>>>>>>> us
> >> >>>>>>>>>>        the header
> >> >>>>>>>>>>        > >>>>>> stuff.
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>> I think the abstraction of always
> >> repartitioning
> >> >>> back is
> >> >>>>>>>>>>        just not so
> >> >>>>>>>>>>        > >>>>>> strong. Like the work has been done before we
> >> >>> partition
> >> >>>>>>>>>>        back and
> >> >>>>>>>>>>        > >>>>>> grouping
> >> >>>>>>>>>>        > >>>>>> by something else afterwards is really
> common.
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >> >>>>>>>>>>        > >>>>>>
> >> >>>>>>>>>>        > >>>>>> Hi Matthias
> >> >>>>>>>>>>        > >>>>>>> Thank you for your feedback, I do appreciate
> >> it!
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> While name spacing would be possible, it
> would
> >> >>> require
> >> >>>>>>>>>> to
> >> >>>>>>>>>>        > deserialize
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
> >> overhead. I
> >> >>> would
> >> >>>>>>>>>>        suggest to
> >> >>>>>>>>>>        > no
> >> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
> >> this
> >> >>>>>>>>>>
> >> >>>>>>>>> becomes a
> >> >>>>>>>
> >> >>>>>>>>        > problem in
> >> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> >> later
> >> >>> on.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Agreed. I will go with using a reserved
> >> string
> >> >>> and
> >> >>>>>>>>>>        document it.
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> My main concern about the design it the type
> >> of
> >> >>> the
> >> >>>>>>>>>>        result KTable:
> >> >>>>>>>>>>        > If
> >> >>>>>>>>>>        > >>>>>>> I
> >> >>>>>>>>>>        > >>>>>>> understood the proposal correctly,
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> In your example, you have table1 and table2
> >> >>> swapped.
> >> >>>>>>>>>>        Here is how it
> >> >>>>>>>>>>        > >>>>>>> works
> >> >>>>>>>>>>        > >>>>>>> currently:
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> 1) table1 has the records that contain the
> >> >>> foreign key
> >> >>>>>>>>>>        within their
> >> >>>>>>>>>>        > >>>>>>> value.
> >> >>>>>>>>>>        > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >> >>>>>>>>>> <b,(fk=A,bar=2)>,
> >> >>>>>>>>>>        > >>>>>>> <c,(fk=B,bar=3)>
> >> >>>>>>>>>>        > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> 2) A Value mapper is required to extract the
> >> >>> foreign
> >> >>>>>>>>>> key.
> >> >>>>>>>>>>        > >>>>>>> table1 foreign key mapper: ( value =>
> >> value.fk
> >> >>>>>>>>>>        <http://value.fk> )
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> The mapper is applied to each element in
> >> table1,
> >> >>> and a
> >> >>>>>>>>>>        new combined
> >> >>>>>>>>>>        > >>>>>>> key is
> >> >>>>>>>>>>        > >>>>>>> made:
> >> >>>>>>>>>>        > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> >> >>>>>>>>>> (fk=A,bar=2)>,
> >> >>>>>>>>>>        <B-c,
> >> >>>>>>>>>>        > >>>>>>> (fk=B,bar=3)>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> 3) The rekeyed events are copartitioned with
> >> >>> table2:
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> a) Stream Thread with Partition 0:
> >> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
> >> <A-b,
> >> >>>>>>>>>>        (fk=A,bar=2)>
> >> >>>>>>>>>>        > >>>>>>> Table2: <A,X>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> b) Stream Thread with Partition 1:
> >> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >> >>>>>>>>>>        > >>>>>>> Table2: <B,Y>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> 4) From here, they can be joined together
> >> locally
> >> >>> by
> >> >>>>>>>>>>        applying the
> >> >>>>>>>>>>        > >>>>>>> joiner
> >> >>>>>>>>>>        > >>>>>>> function.
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> At this point, Jan's design and my design
> >> >>> deviate. My
> >> >>>>>>>>>>        design goes
> >> >>>>>>>>>>        > on
> >> >>>>>>>>>>        > >>>>>>> to
> >> >>>>>>>>>>        > >>>>>>> repartition the data post-join and resolve
> >> >>> out-of-order
> >> >>>>>>>>>>        arrival of
> >> >>>>>>>>>>        > >>>>>>> records,
> >> >>>>>>>>>>        > >>>>>>> finally returning the data keyed just the
> >> >>> original key.
> >> >>>>>>>>>>        I do not
> >> >>>>>>>>>>        > >>>>>>> expose
> >> >>>>>>>>>>        > >>>>>>> the
> >> >>>>>>>>>>        > >>>>>>> CombinedKey or any of the internals outside
> >> of the
> >> >>>>>>>>>>        joinOnForeignKey
> >> >>>>>>>>>>        > >>>>>>> function. This does make for larger
> footprint,
> >> >>> but it
> >> >>>>>>>>>>        removes all
> >> >>>>>>>>>>        > >>>>>>> agency
> >> >>>>>>>>>>        > >>>>>>> for resolving out-of-order arrivals and
> >> handling
> >> >>>>>>>>>>        CombinedKeys from
> >> >>>>>>>>>>        > the
> >> >>>>>>>>>>        > >>>>>>> user. I believe that this makes the function
> >> much
> >> >>>>>>>>>> easier
> >> >>>>>>>>>>        to use.
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> Let me know if this helps resolve your
> >> questions,
> >> >>> and
> >> >>>>>>>>>>        please feel
> >> >>>>>>>>>>        > >>>>>>> free to
> >> >>>>>>>>>>        > >>>>>>> add anything else on your mind.
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> Thanks again,
> >> >>>>>>>>>>        > >>>>>>> Adam
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
> >> Sax <
> >> >>>>>>>>>>        > >>>>>>> matthias@confluent.io <mailto:
> >> >>> matthias@confluent.io>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>> wrote:
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>> Hi,
> >> >>>>>>>>>>        > >>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> I am just catching up on this thread. I did
> >> not
> >> >>> read
> >> >>>>>>>>>>        everything so
> >> >>>>>>>>>>        > >>>>>>>> far,
> >> >>>>>>>>>>        > >>>>>>>> but want to share couple of initial
> thoughts:
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Headers: I think there is a fundamental
> >> >>> difference
> >> >>>>>>>>>>        between header
> >> >>>>>>>>>>        > >>>>>>>> usage
> >> >>>>>>>>>>        > >>>>>>>> in this KIP and KP-258. For 258, we add
> >> headers
> >> >>> to
> >> >>>>>>>>>>        changelog topic
> >> >>>>>>>>>>        > >>>>>>>> that
> >> >>>>>>>>>>        > >>>>>>>> are owned by Kafka Streams and nobody else
> is
> >> >>> supposed
> >> >>>>>>>>>>        to write
> >> >>>>>>>>>>        > into
> >> >>>>>>>>>>        > >>>>>>>> them. In fact, no user header are written
> >> into
> >> >>> the
> >> >>>>>>>>>>        changelog topic
> >> >>>>>>>>>>        > >>>>>>>> and
> >> >>>>>>>>>>        > >>>>>>>> thus, there are not conflicts.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Nevertheless, I don't see a big issue with
> >> using
> >> >>>>>>>>>>        headers within
> >> >>>>>>>>>>        > >>>>>>>> Streams.
> >> >>>>>>>>>>        > >>>>>>>> As long as we document it, we can have some
> >> >>> "reserved"
> >> >>>>>>>>>>        header keys
> >> >>>>>>>>>>        > >>>>>>>> and
> >> >>>>>>>>>>        > >>>>>>>> users are not allowed to use when
> processing
> >> >>> data with
> >> >>>>>>>>>>        Kafka
> >> >>>>>>>>>>        > Streams.
> >> >>>>>>>>>>        > >>>>>>>> IMHO, this should be ok.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> I think there is a safe way to avoid
> >> conflicts,
> >> >>> since
> >> >>>>>>>>>> these
> >> >>>>>>>>>>        > headers
> >> >>>>>>>>>>        > >>>>>>>> are
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>> only needed in internal topics (I think):
> >> >>>>>>>>>>        > >>>>>>>>> For internal and changelog topics, we can
> >> >>> namespace
> >> >>>>>>>>>>        all headers:
> >> >>>>>>>>>>        > >>>>>>>>> * user-defined headers are namespaced as
> >> >>> "external."
> >> >>>>>>>>>> +
> >> >>>>>>>>>>        headerKey
> >> >>>>>>>>>>        > >>>>>>>>> * internal headers are namespaced as
> >> >>> "internal." +
> >> >>>>>>>>>>        headerKey
> >> >>>>>>>>>>        > >>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>> While name spacing would be possible, it
> >> would
> >> >>>>>>>>>> require
> >> >>>>>>>>>>
> >> >>>>>>>>> to
> >> >>>>>>>
> >> >>>>>>>>        > >>>>>>>> deserialize
> >> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
> >> overhead. I
> >> >>> would
> >> >>>>>>>>>>        suggest to
> >> >>>>>>>>>>        > no
> >> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
> >> this
> >> >>>>>>>>>>
> >> >>>>>>>>> becomes a
> >> >>>>>>>
> >> >>>>>>>>        > problem in
> >> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> >> later
> >> >>> on.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> My main concern about the design it the
> type
> >> of
> >> >>> the
> >> >>>>>>>>>>        result KTable:
> >> >>>>>>>>>>        > >>>>>>>> If I
> >> >>>>>>>>>>        > >>>>>>>> understood the proposal correctly,
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> KTable<K1,V1> table1 = ...
> >> >>>>>>>>>>        > >>>>>>>> KTable<K2,V2> table2 = ...
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> KTable<K1,V3> joinedTable =
> >> >>> table1.join(table2,...);
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> implies that the `joinedTable` has the same
> >> key
> >> >>> as the
> >> >>>>>>>>>>        left input
> >> >>>>>>>>>>        > >>>>>>>> table.
> >> >>>>>>>>>>        > >>>>>>>> IMHO, this does not work because if table2
> >> >>> contains
> >> >>>>>>>>>>        multiple rows
> >> >>>>>>>>>>        > >>>>>>>> that
> >> >>>>>>>>>>        > >>>>>>>> join with a record in table1 (what is the
> >> main
> >> >>> purpose
> >> >>>>>>>>>>
> >> >>>>>>>>> of
> >> >>>>>>>
> >> >>>>>>>> a
> >> >>>>>>>>>>        > foreign
> >> >>>>>>>>>>        > >>>>>>>> key
> >> >>>>>>>>>>        > >>>>>>>> join), the result table would only contain
> a
> >> >>> single
> >> >>>>>>>>>>        join result,
> >> >>>>>>>>>>        > but
> >> >>>>>>>>>>        > >>>>>>>> not
> >> >>>>>>>>>>        > >>>>>>>> multiple.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Example:
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> table1 input stream: <A,X>
> >> >>>>>>>>>>        > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> We use table2 value a foreign key to table1
> >> key
> >> >>> (ie,
> >> >>>>>>>>>>        "A" joins).
> >> >>>>>>>>>>        > If
> >> >>>>>>>>>>        > >>>>>>>> the
> >> >>>>>>>>>>        > >>>>>>>> result key is the same key as key of
> table1,
> >> this
> >> >>>>>>>>>>        implies that the
> >> >>>>>>>>>>        > >>>>>>>> result can either be <A, join(X,1)> or <A,
> >> >>> join(X,2)>
> >> >>>>>>>>>>        but not
> >> >>>>>>>>>>        > both.
> >> >>>>>>>>>>        > >>>>>>>> Because the share the same key, whatever
> >> result
> >> >>> record
> >> >>>>>>>>>>        we emit
> >> >>>>>>>>>>        > later,
> >> >>>>>>>>>>        > >>>>>>>> overwrite the previous result.
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> This is the reason why Jan originally
> >> proposed
> >> >>> to use
> >> >>>>>>>>>> a
> >> >>>>>>>>>>        > combination
> >> >>>>>>>>>>        > >>>>>>>> of
> >> >>>>>>>>>>        > >>>>>>>> both primary keys of the input tables as
> key
> >> of
> >> >>> the
> >> >>>>>>>>>>        output table.
> >> >>>>>>>>>>        > >>>>>>>> This
> >> >>>>>>>>>>        > >>>>>>>> makes the keys of the output table unique
> >> and we
> >> >>> can
> >> >>>>>>>>>>        store both in
> >> >>>>>>>>>>        > >>>>>>>> the
> >> >>>>>>>>>>        > >>>>>>>> output table:
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
> >> >>> join(X,2)>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Thoughts?
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> -Matthias
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >> >>>>>>>>>>        > >>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>> Just on remark here.
> >> >>>>>>>>>>        > >>>>>>>>> The high-watermark could be disregarded.
> The
> >> >>> decision
> >> >>>>>>>>>>        about the
> >> >>>>>>>>>>        > >>>>>>>>> forward
> >> >>>>>>>>>>        > >>>>>>>>> depends on the size of the aggregated map.
> >> >>>>>>>>>>        > >>>>>>>>> Only 1 element long maps would be unpacked
> >> and
> >> >>>>>>>>>>        forwarded. 0
> >> >>>>>>>>>>        > element
> >> >>>>>>>>>>        > >>>>>>>>> maps
> >> >>>>>>>>>>        > >>>>>>>>> would be published as delete. Any other
> >> count
> >> >>>>>>>>>>        > >>>>>>>>> of map entries is in "waiting for correct
> >> >>> deletes to
> >> >>>>>>>>>>        > arrive"-state.
> >> >>>>>>>>>>        > >>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >> >>>>>>>>>>        > >>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>> It does look like I could replace the
> second
> >> >>>>>>>>>>        repartition store
> >> >>>>>>>>>>        > and
> >> >>>>>>>>>>        > >>>>>>>>>> highwater store with a groupBy and
> reduce.
> >> >>> However,
> >> >>>>>>>>>>        it looks
> >> >>>>>>>>>>        > like
> >> >>>>>>>>>>        > >>>>>>>>>> I
> >> >>>>>>>>>>        > >>>>>>>>>> would
> >> >>>>>>>>>>        > >>>>>>>>>> still need to store the highwater value
> >> within
> >> >>> the
> >> >>>>>>>>>>        materialized
> >> >>>>>>>>>>        > >>>>>>>>>> store,
> >> >>>>>>>>>>        > >>>>>>>>>>
> >> >>>>>>>>>>        > >>>>>>>>>> to
> >> >>>>>>>>>>        > >>>>>>>>> compare the arrival of out-of-order
> records
> >> >>> (assuming
> >> >>>>>>>>>>
> >> >>>>>>>>> my
> >> >>>>>>>
> >> >>>>>>>>        > >>>>>>>>> understanding
> >> >>>>>>>>>>        > >>>>>>>>> of
> >> >>>>>>>>>>        > >>>>>>>>> THIS is correct...). This in effect is the
> >> same
> >> >>> as
> >> >>>>>>>>>> the
> >> >>>>>>>>>>        design I
> >> >>>>>>>>>>        > have
> >> >>>>>>>>>>        > >>>>>>>>> now,
> >> >>>>>>>>>>        > >>>>>>>>> just with the two tables merged together.
> >> >>>>>>>>>>        > >>>>>>>>>
> >> >>>>>>>>>>        >
> >> >>>>>>>>>>        >
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> --
> >> >>>>>>>> -- Guozhang
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> --
> >> >>>>>>> -- Guozhang
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Actually, I've been thinking more about my feedback #1 on the method name,
and I'm not so sure it's a good idea.

Following SQL, the existing join methods are named according to handedness
(inner/outer/left). All these joins are the same cardinality (1:1 joins). I
think it would be a mistake to switch up the naming scheme and introduce a
new method named for cardinality, like "manyToOneJoin".

Can we actually just add a new overload to the existing joins? The javadoc
could explain that it's a many-to-one join, and it would be differentiated
by the presence of the "keyExtractor".

Just to be safe, if we rename the proposed "joinOnForeignKey" to just
"join", then we could rename "keyExtractor" to "foreignKeyExtractor" for
clarity.

Just to be unambiguous, including the other API feedback I gave, I'm
proposing something like this:

KTable<K, VR> join(KTable<KO, VO> other,
                   ValueMapper<V, KO> foreignKeyExtractor,
                   ValueJoiner<V, VO, VR> joiner,
                   ManyToOneJoined manyToOneJoined, // join config object
                  );

The ManyToOneJoined config object would allow setting all 4 serdes and
configuring materialization.

Thanks for your consideration,
-John

On Thu, Nov 29, 2018 at 8:14 AM John Roesler <jo...@confluent.io> wrote:

> Hi all,
>
> Sorry that this discussion petered out... I think the 2.1 release caused
> an extended distraction that pushed it off everyone's radar (which was
> precisely Adam's concern). Personally, I've also had some extend
> distractions of my own that kept (and continue to keep) me preoccupied.
>
> However, calling for a vote did wake me up, so I guess Jan was on the
> right track!
>
> I've gone back and reviewed the whole KIP document and the prior
> discussion, and I'd like to offer a few thoughts:
>
> API Thoughts:
>
> 1. If I read the KIP right, you are proposing a many-to-one join. Could we
> consider naming it manyToOneJoin? Or, if you prefer, flip the design around
> and make it a oneToManyJoin?
>
> The proposed name "joinOnForeignKey" disguises the join type, and it seems
> like it might trick some people into using it for a one-to-one join. This
> would work, of course, but it would be super inefficient compared to a
> simple rekey-and-join.
>
> 2. I might have missed it, but I don't think it's specified whether it's
> an inner, outer, or left join. I'm guessing an outer join, as (neglecting
> IQ), the rest can be achieved by filtering or by handling it in the
> ValueJoiner.
>
> 3. The arg list to joinOnForeignKey doesn't look quite right.
> 3a. Regarding Serialized: There are a few different paradigms in play in
> the Streams API, so it's confusing, but instead of three Serialized args, I
> think it would be better to have one that allows (optionally) setting the 4
> incoming serdes. The result serde is defined by the Materialized. The
> incoming serdes can be optional because they might already be available on
> the source KTables, or the default serdes from the config might be
> applicable.
>
> 3b. Is the StreamPartitioner necessary? The other joins don't allow
> setting one, and it seems like it might actually be harmful, since the
> rekey operation needs to produce results that are co-partitioned with the
> "other" KTable.
>
> 4. I'm fine with the "reserved word" header, but I didn't actually follow
> what Matthias meant about namespacing requiring "deserializing" the record
> header. The headers are already Strings, so I don't think that
> deserialization is required. If we applied the namespace at source nodes
> and stripped it at sink nodes, this would be practically no overhead. The
> advantage of the namespace idea is that no public API change wrt headers
> needs to happen, and no restrictions need to be placed on users' headers.
>
> (Although I'm wondering if we can get away without the header at all...
> stay tuned)
>
> 5. I also didn't follow the discussion about the HWM table growing without
> bound. As I read it, the HWM table is effectively implementing OCC to
> resolve the problem you noted with disordering when the rekey is
> reversed... particularly notable when the FK changes. As such, it only
> needs to track the most recent "version" (the offset in the source
> partition) of each key. Therefore, it should have the same number of keys
> as the source table at all times.
>
> I see that you are aware of KIP-258, which I think might be relevant in a
> couple of ways. One: it's just about storing the timestamp in the state
> store, but the ultimate idea is to effectively use the timestamp as an OCC
> "version" to drop disordered updates. You wouldn't want to use the
> timestamp for this operation, but if you were to use a similar mechanism to
> store the source offset in the store alongside the re-keyed values, then
> you could avoid a separate table.
>
> 6. You and Jan have been thinking about this for a long time, so I've
> probably missed something here, but I'm wondering if we can avoid the HWM
> tracking at all and resolve out-of-order during a final join instead...
>
> Let's say we're joining a left table (Integer K: Letter FK, (other data))
> to a right table (Letter K: (some data)).
>
> Left table:
> 1: (A, xyz)
> 2: (B, asd)
>
> Right table:
> A: EntityA
> B: EntityB
>
> We could do a rekey as you proposed with a combined key, but not
> propagating the value at all..
> Rekey table:
> A-1: (dummy value)
> B-2: (dummy value)
>
> Which we then join with the right table to produce:
> A-1: EntityA
> B-2: EntityB
>
> Which gets rekeyed back:
> 1: A, EntityA
> 2: B, EntityB
>
> And finally we do the actual join:
> Result table:
> 1: ((A, xyz), EntityA)
> 2: ((B, asd), EntityB)
>
> The thing is that in that last join, we have the opportunity to compare
> the current FK in the left table with the incoming PK of the right table.
> If they don't match, we just drop the event, since it must be outdated.
>
> In your KIP, you gave an example in which (1: A, xyz) gets updated to (1:
> B, xyz), ultimately yielding a conundrum about whether the final state
> should be (1: null) or (1: joined-on-B). With the algorithm above, you
> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> EntityB)). It seems like this does give you enough information to make the
> right choice, regardless of disordering.
>
>
> 7. Last thought... I'm a little concerned about the performance of the
> range scans when records change in the right table. You've said that you've
> been using the algorithm you presented in production for a while. Can you
> give us a sense of the performance characteristics you've observed?
>
> I could only think of one alternative, but I'm not sure if it's better or
> worse... If the first re-key only needs to preserve the original key, as I
> proposed in #6, then we could store a vector of keys in the value:
>
> Left table:
> 1: A,...
> 2: B,...
> 3: A,...
>
> Gets re-keyed:
> A: [1, 3]
> B: [2]
>
> Then, the rhs part of the join would only need a regular single-key
> lookup. Of course we have to deal with the problem of large values, as
> there's no bound on the number of lhs records that can reference rhs
> records. Offhand, I'd say we could page the values, so when one row is past
> the threshold, we append the key for the next page. Then in most cases, it
> would be a single key lookup, but for large fan-out updates, it would be
> one per (max value size)/(avg lhs key size).
>
> This seems more complex, though... Plus, I think there's some extra
> tracking we'd need to do to know when to emit a retraction. For example,
> when record 1 is deleted, the re-key table would just have (A: [3]). Some
> kind of tombstone is needed so that the join result for 1 can also be
> retracted.
>
> That's all!
>
> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> discussion has been slow.
> -John
>
>
> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Id say you can just call the vote.
>>
>> that happens all the time, and if something comes up, it just goes back
>> to discuss.
>>
>> would not expect to much attention with another another email in this
>> thread.
>>
>> best Jan
>>
>> On 09.10.2018 13:56, Adam Bellemare wrote:
>> > Hello Contributors
>> >
>> > I know that 2.1 is about to be released, but I do need to bump this to
>> keep
>> > visibility up. I am still intending to push this through once
>> contributor
>> > feedback is given.
>> >
>> > Main points that need addressing:
>> > 1) Any way (or benefit) in structuring the current singular graph node
>> into
>> > multiple nodes? It has a whopping 25 parameters right now. I am a bit
>> fuzzy
>> > on how the optimizations are supposed to work, so I would appreciate any
>> > help on this aspect.
>> >
>> > 2) Overall strategy for joining + resolving. This thread has much
>> discourse
>> > between Jan and I between the current highwater mark proposal and a
>> groupBy
>> > + reduce proposal. I am of the opinion that we need to strictly handle
>> any
>> > chance of out-of-order data and leave none of it up to the consumer. Any
>> > comments or suggestions here would also help.
>> >
>> > 3) Anything else that you see that would prevent this from moving to a
>> vote?
>> >
>> > Thanks
>> >
>> > Adam
>> >
>> >
>> >
>> >
>> >
>> >
>> >
>> > On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>> > wrote:
>> >
>> >> Hi Jan
>> >>
>> >> With the Stores.windowStoreBuilder and Stores.persistentWindowStore,
>> you
>> >> actually only need to specify the amount of segments you want and how
>> large
>> >> they are. To the best of my understanding, what happens is that the
>> >> segments are automatically rolled over as new data with new timestamps
>> are
>> >> created. We use this exact functionality in some of the work done
>> >> internally at my company. For reference, this is the hopping windowed
>> store.
>> >>
>> >>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>> >>
>> >> In the code that I have provided, there are going to be two 24h
>> segments.
>> >> When a record is put into the windowStore, it will be inserted at time
>> T in
>> >> both segments. The two segments will always overlap by 12h. As time
>> goes on
>> >> and new records are added (say at time T+12h+), the oldest segment
>> will be
>> >> automatically deleted and a new segment created. The records are by
>> default
>> >> inserted with the context.timestamp(), such that it is the record
>> time, not
>> >> the clock time, which is used.
>> >>
>> >> To the best of my understanding, the timestamps are retained when
>> >> restoring from the changelog.
>> >>
>> >> Basically, this is heavy-handed way to deal with TTL at a
>> segment-level,
>> >> instead of at an individual record level.
>> >>
>> >> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Jan.Filipiak@trivago.com
>> >
>> >> wrote:
>> >>
>> >>> Will that work? I expected it to blow up with ClassCastException or
>> >>> similar.
>> >>>
>> >>> You either would have to specify the window you fetch/put or iterate
>> >>> across all windows the key was found in right?
>> >>>
>> >>> I just hope the window-store doesn't check stream-time under the hoods
>> >>> that would be a questionable interface.
>> >>>
>> >>> If it does: did you see my comment on checking all the windows
>> earlier?
>> >>> that would be needed to actually give reasonable time gurantees.
>> >>>
>> >>> Best
>> >>>
>> >>>
>> >>>
>> >>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> >>>> Hi Jan
>> >>>>
>> >>>> Check for  " highwaterMat " in the PR. I only changed the state
>> store,
>> >>> not
>> >>>> the ProcessorSupplier.
>> >>>>
>> >>>> Thanks,
>> >>>> Adam
>> >>>>
>> >>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> Jan.Filipiak@trivago.com
>> >>>>
>> >>>> wrote:
>> >>>>
>> >>>>>
>> >>>>>
>> >>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>> >>>>>
>> >>>>>> @Guozhang
>> >>>>>>
>> >>>>>> Thanks for the information. This is indeed something that will be
>> >>>>>> extremely
>> >>>>>> useful for this KIP.
>> >>>>>>
>> >>>>>> @Jan
>> >>>>>> Thanks for your explanations. That being said, I will not be moving
>> >>> ahead
>> >>>>>> with an implementation using reshuffle/groupBy solution as you
>> >>> propose.
>> >>>>>> That being said, if you wish to implement it yourself off of my
>> >>> current PR
>> >>>>>> and submit it as a competitive alternative, I would be more than
>> >>> happy to
>> >>>>>> help vet that as an alternate solution. As it stands right now, I
>> do
>> >>> not
>> >>>>>> really have more time to invest into alternatives without there
>> being
>> >>> a
>> >>>>>> strong indication from the binding voters which they would prefer.
>> >>>>>>
>> >>>>>>
>> >>>>> Hey, total no worries. I think I personally gave up on the streams
>> DSL
>> >>> for
>> >>>>> some time already, otherwise I would have pulled this KIP through
>> >>> already.
>> >>>>> I am currently reimplementing my own DSL based on PAPI.
>> >>>>>
>> >>>>>
>> >>>>>> I will look at finishing up my PR with the windowed state store in
>> the
>> >>>>>> next
>> >>>>>> week or so, exercising it via tests, and then I will come back for
>> >>> final
>> >>>>>> discussions. In the meantime, I hope that any of the binding voters
>> >>> could
>> >>>>>> take a look at the KIP in the wiki. I have updated it according to
>> the
>> >>>>>> latest plan:
>> >>>>>>
>> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> >>>>>> Support+non-key+joining+in+KTable
>> >>>>>>
>> >>>>>> I have also updated the KIP PR to use a windowed store. This could
>> be
>> >>>>>> replaced by the results of KIP-258 whenever they are completed.
>> >>>>>> https://github.com/apache/kafka/pull/5527
>> >>>>>>
>> >>>>>> Thanks,
>> >>>>>>
>> >>>>>> Adam
>> >>>>>>
>> >>>>>
>> >>>>> Is the HighWatermarkResolverProccessorsupplier already updated in
>> the
>> >>> PR?
>> >>>>> expected it to change to Windowed<K>,Long Missing something?
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as
>> it
>> >>> is
>> >>>>>>> for
>> >>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
>> >>> want to
>> >>>>>>> have "handling out-of-order data for source KTable" such that
>> >>> instead of
>> >>>>>>> blindly apply the updates to the materialized store, i.e.
>> following
>> >>>>>>> offset
>> >>>>>>> ordering, we will reject updates that are older than the current
>> >>> key's
>> >>>>>>> timestamps, i.e. following timestamp ordering.
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> Guozhang
>> >>>>>>>
>> >>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>> wangguoz@gmail.com>
>> >>>>>>> wrote:
>> >>>>>>>
>> >>>>>>> Hello Adam,
>> >>>>>>>>
>> >>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
>> high
>> >>>>>>>> watermark store, now altered to be replaced with a window
>> store), I
>> >>>>>>>> think
>> >>>>>>>> another current on-going KIP may actually help:
>> >>>>>>>>
>> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> This is for adding the timestamp into a key-value store (i.e.
>> only
>> >>> for
>> >>>>>>>> non-windowed KTable), and then one of its usage, as described in
>> >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
>> >>> then
>> >>>>>>>> "reject" updates from the source topics if its timestamp is
>> smaller
>> >>> than
>> >>>>>>>> the current key's latest update timestamp. I think it is very
>> >>> similar to
>> >>>>>>>> what you have in mind for high watermark based filtering, while
>> you
>> >>> only
>> >>>>>>>> need to make sure that the timestamps of the joining records are
>> >>>>>>>>
>> >>>>>>> correctly
>> >>>>>>>
>> >>>>>>>> inherited though the whole topology to the final stage.
>> >>>>>>>>
>> >>>>>>>> Note that this KIP is for key-value store and hence non-windowed
>> >>> KTables
>> >>>>>>>> only, but for windowed KTables we do not really have a good
>> support
>> >>> for
>> >>>>>>>> their joins anyways (
>> >>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> >>>>>>>> I
>> >>>>>>>> think we can just consider non-windowed KTable-KTable non-key
>> joins
>> >>> for
>> >>>>>>>> now. In which case, KIP-258 should help.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>> >>> Jan.Filipiak@trivago.com
>> >>>>>>>>>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> >>>>>>>>>
>> >>>>>>>>> Hi Guozhang
>> >>>>>>>>>>
>> >>>>>>>>>> Current highwater mark implementation would grow endlessly
>> based
>> >>> on
>> >>>>>>>>>> primary key of original event. It is a pair of (<this table
>> >>> primary
>> >>>>>>>>>>
>> >>>>>>>>> key>,
>> >>>>>>>
>> >>>>>>>> <highest offset seen for that key>). This is used to
>> differentiate
>> >>>>>>>>>>
>> >>>>>>>>> between
>> >>>>>>>
>> >>>>>>>> late arrivals and new updates. My newest proposal would be to
>> >>> replace
>> >>>>>>>>>>
>> >>>>>>>>> it
>> >>>>>>>
>> >>>>>>>> with a Windowed state store of Duration N. This would allow the
>> same
>> >>>>>>>>>> behaviour, but cap the size based on time. This should allow
>> for
>> >>> all
>> >>>>>>>>>> late-arriving events to be processed, and should be
>> customizable
>> >>> by
>> >>>>>>>>>> the
>> >>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes
>> of
>> >>>>>>>>>>
>> >>>>>>>>> window,
>> >>>>>>>
>> >>>>>>>> or perhaps 7 days...).
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Adam, using time based retention can do the trick here. Even
>> >>> if I
>> >>>>>>>>> would still like to see the automatic repartitioning optional
>> >>> since I
>> >>>>>>>>>
>> >>>>>>>> would
>> >>>>>>>
>> >>>>>>>> just reshuffle again. With windowed store I am a little bit
>> >>> sceptical
>> >>>>>>>>>
>> >>>>>>>> about
>> >>>>>>>
>> >>>>>>>> how to determine the window. So esentially one could run into
>> >>> problems
>> >>>>>>>>>
>> >>>>>>>> when
>> >>>>>>>
>> >>>>>>>> the rapid change happens near a window border. I will check you
>> >>>>>>>>> implementation in detail, if its problematic, we could still
>> check
>> >>>>>>>>> _all_
>> >>>>>>>>> windows on read with not to bad performance impact I guess. Will
>> >>> let
>> >>>>>>>>> you
>> >>>>>>>>> know if the implementation would be correct as is. I wouldn't
>> not
>> >>> like
>> >>>>>>>>>
>> >>>>>>>> to
>> >>>>>>>
>> >>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>> timestamp(B).
>> >>> I
>> >>>>>>>>>
>> >>>>>>>> think
>> >>>>>>>
>> >>>>>>>> we can't expect that.
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> @Jan
>> >>>>>>>>>> I believe I understand what you mean now - thanks for the
>> >>> diagram, it
>> >>>>>>>>>> did really help. You are correct that I do not have the
>> original
>> >>>>>>>>>>
>> >>>>>>>>> primary
>> >>>>>>>
>> >>>>>>>> key available, and I can see that if it was available then you
>> >>> would be
>> >>>>>>>>>> able to add and remove events from the Map. That being said, I
>> >>>>>>>>>>
>> >>>>>>>>> encourage
>> >>>>>>>
>> >>>>>>>> you to finish your diagrams / charts just for clarity for
>> everyone
>> >>>>>>>>>>
>> >>>>>>>>> else.
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
>> >>> understand
>> >>>>>>>>>>
>> >>>>>>>>> the benefits for the rest. Sorry about the original primary
>> key, We
>> >>>>>>>>> have
>> >>>>>>>>> join and Group by implemented our own in PAPI and basically not
>> >>> using
>> >>>>>>>>>
>> >>>>>>>> any
>> >>>>>>>
>> >>>>>>>> DSL (Just the abstraction). Completely missed that in original
>> DSL
>> >>> its
>> >>>>>>>>>
>> >>>>>>>> not
>> >>>>>>>
>> >>>>>>>> there and just assumed it. total brain mess up on my end. Will
>> >>> finish
>> >>>>>>>>>
>> >>>>>>>> the
>> >>>>>>>
>> >>>>>>>> chart as soon as i get a quite evening this week.
>> >>>>>>>>>
>> >>>>>>>>> My follow up question for you is, won't the Map stay inside the
>> >>> State
>> >>>>>>>>>
>> >>>>>>>>>> Store indefinitely after all of the changes have propagated?
>> Isn't
>> >>>>>>>>>> this
>> >>>>>>>>>> effectively the same as a highwater mark state store?
>> >>>>>>>>>>
>> >>>>>>>>>> Thing is that if the map is empty, substractor is gonna return
>> >>> `null`
>> >>>>>>>>>
>> >>>>>>>> and
>> >>>>>>>
>> >>>>>>>> the key is removed from the keyspace. But there is going to be a
>> >>> store
>> >>>>>>>>> 100%, the good thing is that I can use this store directly for
>> >>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
>> >>> satisfying
>> >>>>>>>>> all gurantees needed for further groupby / join. The Windowed
>> >>> store is
>> >>>>>>>>>
>> >>>>>>>> not
>> >>>>>>>
>> >>>>>>>> keeping the values, so for the next statefull operation we would
>> >>>>>>>>> need to instantiate an extra store. or we have the window store
>> >>> also
>> >>>>>>>>>
>> >>>>>>>> have
>> >>>>>>>
>> >>>>>>>> the values then.
>> >>>>>>>>>
>> >>>>>>>>> Long story short. if we can flip in a custom group by before
>> >>>>>>>>> repartitioning to the original primary key i think it would help
>> >>> the
>> >>>>>>>>>
>> >>>>>>>> users
>> >>>>>>>
>> >>>>>>>> big time in building efficient apps. Given the original primary
>> key
>> >>>>>>>>>
>> >>>>>>>> issue I
>> >>>>>>>
>> >>>>>>>> understand that we do not have a solid foundation to build on.
>> >>>>>>>>> Leaving primary key carry along to the user. very unfortunate. I
>> >>> could
>> >>>>>>>>> understand the decision goes like that. I do not think its a
>> good
>> >>>>>>>>>
>> >>>>>>>> decision.
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks
>> >>>>>>>>>> Adam
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>> >>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>> dumbreprajakta311@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>> wrote:
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>>>        please remove me from this group
>> >>>>>>>>>>
>> >>>>>>>>>>        On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>> >>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>        wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>        > Hi Adam,
>> >>>>>>>>>>        >
>> >>>>>>>>>>        > give me some time, will make such a chart. last time i
>> >>> didn't
>> >>>>>>>>>>        get along
>> >>>>>>>>>>        > well with giphy and ruined all your charts.
>> >>>>>>>>>>        > Hopefully i can get it done today
>> >>>>>>>>>>        >
>> >>>>>>>>>>        > On 08.09.2018 16:00, Adam Bellemare wrote:
>> >>>>>>>>>>        > > Hi Jan
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > > I have included a diagram of what I attempted on the
>> >>> KIP.
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        >
>> >>>>>>>>>>
>> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> >>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> >>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> >>>>>>>>>>        <
>> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> >>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> >>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > > I attempted this back at the start of my own
>> >>> implementation
>> >>>>>>>>>> of
>> >>>>>>>>>>        this
>> >>>>>>>>>>        > > solution, and since I could not get it to work I
>> have
>> >>> since
>> >>>>>>>>>>        discarded the
>> >>>>>>>>>>        > > code. At this point in time, if you wish to continue
>> >>> pursuing
>> >>>>>>>>>>        for your
>> >>>>>>>>>>        > > groupBy solution, I ask that you please create a
>> >>> diagram on
>> >>>>>>>>>>        the KIP
>> >>>>>>>>>>        > > carefully explaining your solution. Please feel
>> free to
>> >>> use
>> >>>>>>>>>>        the image I
>> >>>>>>>>>>        > > just posted as a starting point. I am having trouble
>> >>>>>>>>>>        understanding your
>> >>>>>>>>>>        > > explanations but I think that a carefully
>> constructed
>> >>> diagram
>> >>>>>>>>>>        will clear
>> >>>>>>>>>>        > up
>> >>>>>>>>>>        > > any misunderstandings. Alternately, please post a
>> >>>>>>>>>>        comprehensive PR with
>> >>>>>>>>>>        > > your solution. I can only guess at what you mean,
>> and
>> >>> since I
>> >>>>>>>>>>        value my
>> >>>>>>>>>>        > own
>> >>>>>>>>>>        > > time as much as you value yours, I believe it is
>> your
>> >>>>>>>>>>        responsibility to
>> >>>>>>>>>>        > > provide an implementation instead of me trying to
>> guess.
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > > Adam
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>> >>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>        > > wrote:
>> >>>>>>>>>>        > >
>> >>>>>>>>>>        > >> Hi James,
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >> nice to see you beeing interested. kafka streams at
>> >>> this
>> >>>>>>>>>>        point supports
>> >>>>>>>>>>        > >> all sorts of joins as long as both streams have the
>> >>> same
>> >>>>>>>>>> key.
>> >>>>>>>>>>        > >> Adam is currently implementing a join where a
>> KTable
>> >>> and a
>> >>>>>>>>>>        KTable can
>> >>>>>>>>>>        > have
>> >>>>>>>>>>        > >> a one to many relation ship (1:n). We exploit that
>> >>> rocksdb
>> >>>>>>>>>> is
>> >>>>>>>>>>
>> >>>>>>>>> a
>> >>>>>>>
>> >>>>>>>>        > >> datastore that keeps data sorted (At least exposes an
>> >>> API to
>> >>>>>>>>>>        access the
>> >>>>>>>>>>        > >> stored data in a sorted fashion).
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >> I think the technical caveats are well understood
>> now
>> >>> and we
>> >>>>>>>>>>
>> >>>>>>>>> are
>> >>>>>>>
>> >>>>>>>>        > basically
>> >>>>>>>>>>        > >> down to philosophy and API Design ( when Adam sees
>> my
>> >>> newest
>> >>>>>>>>>>        message).
>> >>>>>>>>>>        > >> I have a lengthy track record of loosing those
>> kinda
>> >>>>>>>>>>        arguments within
>> >>>>>>>>>>        > the
>> >>>>>>>>>>        > >> streams community and I have no clue why. So I
>> >>> literally
>> >>>>>>>>>>        can't wait for
>> >>>>>>>>>>        > you
>> >>>>>>>>>>        > >> to churn through this thread and give you opinion
>> on
>> >>> how we
>> >>>>>>>>>>        should
>> >>>>>>>>>>        > design
>> >>>>>>>>>>        > >> the return type of the oneToManyJoin and how many
>> >>> power we
>> >>>>>>>>>>        want to give
>> >>>>>>>>>>        > to
>> >>>>>>>>>>        > >> the user vs "simplicity" (where simplicity isn't
>> >>> really that
>> >>>>>>>>>>        as users
>> >>>>>>>>>>        > still
>> >>>>>>>>>>        > >> need to understand it I argue)
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >> waiting for you to join in on the discussion
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >> Best Jan
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >> On 07.09.2018 15:49, James Kwan wrote:
>> >>>>>>>>>>        > >>
>> >>>>>>>>>>        > >>> I am new to this group and I found this subject
>> >>>>>>>>>>        interesting.  Sounds
>> >>>>>>>>>>        > like
>> >>>>>>>>>>        > >>> you guys want to implement a join table of two
>> >>> streams? Is
>> >>>>>>>>>> there
>> >>>>>>>>>>        > somewhere
>> >>>>>>>>>>        > >>> I can see the original requirement or proposal?
>> >>>>>>>>>>        > >>>
>> >>>>>>>>>>        > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>> >>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>        > >>>> wrote:
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>> I'm currently testing using a Windowed Store to
>> >>> store the
>> >>>>>>>>>>        highwater
>> >>>>>>>>>>        > >>>>> mark.
>> >>>>>>>>>>        > >>>>> By all indications this should work fine, with
>> the
>> >>> caveat
>> >>>>>>>>>>        being that
>> >>>>>>>>>>        > it
>> >>>>>>>>>>        > >>>>> can
>> >>>>>>>>>>        > >>>>> only resolve out-of-order arrival for up to the
>> >>> size of
>> >>>>>>>>>>        the window
>> >>>>>>>>>>        > (ie:
>> >>>>>>>>>>        > >>>>> 24h, 72h, etc). This would remove the
>> possibility
>> >>> of it
>> >>>>>>>>>>
>> >>>>>>>>> being
>> >>>>>>>
>> >>>>>>>>        > unbounded
>> >>>>>>>>>>        > >>>>> in
>> >>>>>>>>>>        > >>>>> size.
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>> With regards to Jan's suggestion, I believe
>> this is
>> >>> where
>> >>>>>>>>>>        we will
>> >>>>>>>>>>        > have
>> >>>>>>>>>>        > >>>>> to
>> >>>>>>>>>>        > >>>>> remain in disagreement. While I do not disagree
>> >>> with your
>> >>>>>>>>>>        statement
>> >>>>>>>>>>        > >>>>> about
>> >>>>>>>>>>        > >>>>> there likely to be additional joins done in a
>> >>> real-world
>> >>>>>>>>>>        workflow, I
>> >>>>>>>>>>        > do
>> >>>>>>>>>>        > >>>>> not
>> >>>>>>>>>>        > >>>>> see how you can conclusively deal with
>> out-of-order
>> >>>>>>>>>> arrival
>> >>>>>>>>>> of
>> >>>>>>>>>>        > >>>>> foreign-key
>> >>>>>>>>>>        > >>>>> changes and subsequent joins. I have attempted
>> what
>> >>> I
>> >>>>>>>>>>        think you have
>> >>>>>>>>>>        > >>>>> proposed (without a high-water, using groupBy
>> and
>> >>> reduce)
>> >>>>>>>>>>        and found
>> >>>>>>>>>>        > >>>>> that if
>> >>>>>>>>>>        > >>>>> the foreign key changes too quickly, or the
>> load on
>> >>> a
>> >>>>>>>>>>        stream thread
>> >>>>>>>>>>        > is
>> >>>>>>>>>>        > >>>>> too
>> >>>>>>>>>>        > >>>>> high, the joined messages will arrive
>> out-of-order
>> >>> and be
>> >>>>>>>>>>        incorrectly
>> >>>>>>>>>>        > >>>>> propagated, such that an intermediate event is
>> >>>>>>>>>> represented
>> >>>>>>>>>>        as the
>> >>>>>>>>>>        > final
>> >>>>>>>>>>        > >>>>> event.
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>> Can you shed some light on your groupBy
>> >>> implementation.
>> >>>>>>>>>>        There must be
>> >>>>>>>>>>        > >>>> some sort of flaw in it.
>> >>>>>>>>>>        > >>>> I have a suspicion where it is, I would just
>> like to
>> >>>>>>>>>>        confirm. The idea
>> >>>>>>>>>>        > >>>> is bullet proof and it must be
>> >>>>>>>>>>        > >>>> an implementation mess up. I would like to
>> clarify
>> >>> before
>> >>>>>>>>>>        we draw a
>> >>>>>>>>>>        > >>>> conclusion.
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>    Repartitioning the scattered events back to
>> their
>> >>>>>>>>>>
>> >>>>>>>>> original
>> >>>>>>>
>> >>>>>>>>        > >>>>> partitions is the only way I know how to
>> conclusively
>> >>> deal
>> >>>>>>>>>>        with
>> >>>>>>>>>>        > >>>>> out-of-order events in a given time frame, and
>> to
>> >>> ensure
>> >>>>>>>>>>        that the
>> >>>>>>>>>>        > data
>> >>>>>>>>>>        > >>>>> is
>> >>>>>>>>>>        > >>>>> eventually consistent with the input events.
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>> If you have some code to share that illustrates
>> your
>> >>>>>>>>>>        approach, I
>> >>>>>>>>>>        > would
>> >>>>>>>>>>        > >>>>> be
>> >>>>>>>>>>        > >>>>> very grateful as it would remove any
>> >>> misunderstandings
>> >>>>>>>>>>        that I may
>> >>>>>>>>>>        > have.
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>> ah okay you were looking for my code. I don't
>> have
>> >>>>>>>>>>        something easily
>> >>>>>>>>>>        > >>>> readable here as its bloated with OO-patterns.
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>> its anyhow trivial:
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>> @Override
>> >>>>>>>>>>        > >>>>      public T apply(K aggKey, V value, T
>> aggregate)
>> >>>>>>>>>>        > >>>>      {
>> >>>>>>>>>>        > >>>>          Map<U, V> currentStateAsMap =
>> >>> asMap(aggregate);
>> >>>>>>>>>> <<
>> >>>>>>>>>>        imaginary
>> >>>>>>>>>>        > >>>>          U toModifyKey = mapper.apply(value);
>> >>>>>>>>>>        > >>>>              << this is the place where people
>> >>> actually
>> >>>>>>>>>>        gonna have
>> >>>>>>>>>>        > issues
>> >>>>>>>>>>        > >>>> and why you probably couldn't do it. we would
>> need
>> >>> to find
>> >>>>>>>>>>        a solution
>> >>>>>>>>>>        > here.
>> >>>>>>>>>>        > >>>> I didn't realize that yet.
>> >>>>>>>>>>        > >>>>              << we propagate the field in the
>> >>> joiner, so
>> >>>>>>>>>>        that we can
>> >>>>>>>>>>        > pick
>> >>>>>>>>>>        > >>>> it up in an aggregate. Probably you have not
>> thought
>> >>> of
>> >>>>>>>>>>        this in your
>> >>>>>>>>>>        > >>>> approach right?
>> >>>>>>>>>>        > >>>>              << I am very open to find a generic
>> >>> solution
>> >>>>>>>>>>        here. In my
>> >>>>>>>>>>        > >>>> honest opinion this is broken in
>> KTableImpl.GroupBy
>> >>> that
>> >>>>>>>>>> it
>> >>>>>>>>>>        looses
>> >>>>>>>>>>        > the keys
>> >>>>>>>>>>        > >>>> and only maintains the aggregate key.
>> >>>>>>>>>>        > >>>>              << I abstracted it away back then
>> way
>> >>> before
>> >>>>>>>>>> i
>> >>>>>>>>>> was
>> >>>>>>>>>>        > thinking
>> >>>>>>>>>>        > >>>> of oneToMany join. That is why I didn't realize
>> its
>> >>>>>>>>>>        significance here.
>> >>>>>>>>>>        > >>>>              << Opinions?
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>          for (V m : current)
>> >>>>>>>>>>        > >>>>          {
>> >>>>>>>>>>        > >>>> currentStateAsMap.put(mapper.apply(m), m);
>> >>>>>>>>>>        > >>>>          }
>> >>>>>>>>>>        > >>>>          if (isAdder)
>> >>>>>>>>>>        > >>>>          {
>> >>>>>>>>>>        > >>>> currentStateAsMap.put(toModifyKey, value);
>> >>>>>>>>>>        > >>>>          }
>> >>>>>>>>>>        > >>>>          else
>> >>>>>>>>>>        > >>>>          {
>> >>>>>>>>>>        > >>>> currentStateAsMap.remove(toModifyKey);
>> >>>>>>>>>>        > >>>> if(currentStateAsMap.isEmpty()){
>> >>>>>>>>>>        > >>>>                  return null;
>> >>>>>>>>>>        > >>>>              }
>> >>>>>>>>>>        > >>>>          }
>> >>>>>>>>>>        > >>>>          retrun
>> asAggregateType(currentStateAsMap)
>> >>>>>>>>>>        > >>>>      }
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>>
>> >>>>>>>>>>        > >>>> Thanks,
>> >>>>>>>>>>        > >>>>> Adam
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>> >>>>>>>>>>        > Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>> >>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>        > >>>>> wrote:
>> >>>>>>>>>>        > >>>>>
>> >>>>>>>>>>        > >>>>> Thanks Adam for bringing Matthias to speed!
>> >>>>>>>>>>        > >>>>>> about the differences. I think re-keying back
>> >>> should be
>> >>>>>>>>>>        optional at
>> >>>>>>>>>>        > >>>>>> best.
>> >>>>>>>>>>        > >>>>>> I would say we return a KScatteredTable with
>> >>> reshuffle()
>> >>>>>>>>>>        returning
>> >>>>>>>>>>        > >>>>>> KTable<originalKey,Joined> to make the
>> backwards
>> >>>>>>>>>>        repartitioning
>> >>>>>>>>>>        > >>>>>> optional.
>> >>>>>>>>>>        > >>>>>> I am also in a big favour of doing the out of
>> order
>> >>>>>>>>>>        processing using
>> >>>>>>>>>>        > >>>>>> group
>> >>>>>>>>>>        > >>>>>> by instead high water mark tracking.
>> >>>>>>>>>>        > >>>>>> Just because unbounded growth is just scary +
>> It
>> >>> saves
>> >>>>>>>>>> us
>> >>>>>>>>>>        the header
>> >>>>>>>>>>        > >>>>>> stuff.
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>> I think the abstraction of always
>> repartitioning
>> >>> back is
>> >>>>>>>>>>        just not so
>> >>>>>>>>>>        > >>>>>> strong. Like the work has been done before we
>> >>> partition
>> >>>>>>>>>>        back and
>> >>>>>>>>>>        > >>>>>> grouping
>> >>>>>>>>>>        > >>>>>> by something else afterwards is really common.
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>> >>>>>>>>>>        > >>>>>>
>> >>>>>>>>>>        > >>>>>> Hi Matthias
>> >>>>>>>>>>        > >>>>>>> Thank you for your feedback, I do appreciate
>> it!
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> While name spacing would be possible, it would
>> >>> require
>> >>>>>>>>>> to
>> >>>>>>>>>>        > deserialize
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
>> overhead. I
>> >>> would
>> >>>>>>>>>>        suggest to
>> >>>>>>>>>>        > no
>> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
>> this
>> >>>>>>>>>>
>> >>>>>>>>> becomes a
>> >>>>>>>
>> >>>>>>>>        > problem in
>> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
>> later
>> >>> on.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Agreed. I will go with using a reserved
>> string
>> >>> and
>> >>>>>>>>>>        document it.
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> My main concern about the design it the type
>> of
>> >>> the
>> >>>>>>>>>>        result KTable:
>> >>>>>>>>>>        > If
>> >>>>>>>>>>        > >>>>>>> I
>> >>>>>>>>>>        > >>>>>>> understood the proposal correctly,
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> In your example, you have table1 and table2
>> >>> swapped.
>> >>>>>>>>>>        Here is how it
>> >>>>>>>>>>        > >>>>>>> works
>> >>>>>>>>>>        > >>>>>>> currently:
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> 1) table1 has the records that contain the
>> >>> foreign key
>> >>>>>>>>>>        within their
>> >>>>>>>>>>        > >>>>>>> value.
>> >>>>>>>>>>        > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>> >>>>>>>>>> <b,(fk=A,bar=2)>,
>> >>>>>>>>>>        > >>>>>>> <c,(fk=B,bar=3)>
>> >>>>>>>>>>        > >>>>>>> table2 input stream: <A,X>, <B,Y>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> 2) A Value mapper is required to extract the
>> >>> foreign
>> >>>>>>>>>> key.
>> >>>>>>>>>>        > >>>>>>> table1 foreign key mapper: ( value =>
>> value.fk
>> >>>>>>>>>>        <http://value.fk> )
>> >>>>>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> The mapper is applied to each element in
>> table1,
>> >>> and a
>> >>>>>>>>>>        new combined
>> >>>>>>>>>>        > >>>>>>> key is
>> >>>>>>>>>>        > >>>>>>> made:
>> >>>>>>>>>>        > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>> >>>>>>>>>> (fk=A,bar=2)>,
>> >>>>>>>>>>        <B-c,
>> >>>>>>>>>>        > >>>>>>> (fk=B,bar=3)>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> 3) The rekeyed events are copartitioned with
>> >>> table2:
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> a) Stream Thread with Partition 0:
>> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
>> <A-b,
>> >>>>>>>>>>        (fk=A,bar=2)>
>> >>>>>>>>>>        > >>>>>>> Table2: <A,X>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> b) Stream Thread with Partition 1:
>> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>> >>>>>>>>>>        > >>>>>>> Table2: <B,Y>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> 4) From here, they can be joined together
>> locally
>> >>> by
>> >>>>>>>>>>        applying the
>> >>>>>>>>>>        > >>>>>>> joiner
>> >>>>>>>>>>        > >>>>>>> function.
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> At this point, Jan's design and my design
>> >>> deviate. My
>> >>>>>>>>>>        design goes
>> >>>>>>>>>>        > on
>> >>>>>>>>>>        > >>>>>>> to
>> >>>>>>>>>>        > >>>>>>> repartition the data post-join and resolve
>> >>> out-of-order
>> >>>>>>>>>>        arrival of
>> >>>>>>>>>>        > >>>>>>> records,
>> >>>>>>>>>>        > >>>>>>> finally returning the data keyed just the
>> >>> original key.
>> >>>>>>>>>>        I do not
>> >>>>>>>>>>        > >>>>>>> expose
>> >>>>>>>>>>        > >>>>>>> the
>> >>>>>>>>>>        > >>>>>>> CombinedKey or any of the internals outside
>> of the
>> >>>>>>>>>>        joinOnForeignKey
>> >>>>>>>>>>        > >>>>>>> function. This does make for larger footprint,
>> >>> but it
>> >>>>>>>>>>        removes all
>> >>>>>>>>>>        > >>>>>>> agency
>> >>>>>>>>>>        > >>>>>>> for resolving out-of-order arrivals and
>> handling
>> >>>>>>>>>>        CombinedKeys from
>> >>>>>>>>>>        > the
>> >>>>>>>>>>        > >>>>>>> user. I believe that this makes the function
>> much
>> >>>>>>>>>> easier
>> >>>>>>>>>>        to use.
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> Let me know if this helps resolve your
>> questions,
>> >>> and
>> >>>>>>>>>>        please feel
>> >>>>>>>>>>        > >>>>>>> free to
>> >>>>>>>>>>        > >>>>>>> add anything else on your mind.
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> Thanks again,
>> >>>>>>>>>>        > >>>>>>> Adam
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
>> Sax <
>> >>>>>>>>>>        > >>>>>>> matthias@confluent.io <mailto:
>> >>> matthias@confluent.io>>
>> >>>>>>>>>>
>> >>>>>>>>>>        > >>>>>>> wrote:
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>> Hi,
>> >>>>>>>>>>        > >>>>>>>
>> >>>>>>>>>>        > >>>>>>>> I am just catching up on this thread. I did
>> not
>> >>> read
>> >>>>>>>>>>        everything so
>> >>>>>>>>>>        > >>>>>>>> far,
>> >>>>>>>>>>        > >>>>>>>> but want to share couple of initial thoughts:
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Headers: I think there is a fundamental
>> >>> difference
>> >>>>>>>>>>        between header
>> >>>>>>>>>>        > >>>>>>>> usage
>> >>>>>>>>>>        > >>>>>>>> in this KIP and KP-258. For 258, we add
>> headers
>> >>> to
>> >>>>>>>>>>        changelog topic
>> >>>>>>>>>>        > >>>>>>>> that
>> >>>>>>>>>>        > >>>>>>>> are owned by Kafka Streams and nobody else is
>> >>> supposed
>> >>>>>>>>>>        to write
>> >>>>>>>>>>        > into
>> >>>>>>>>>>        > >>>>>>>> them. In fact, no user header are written
>> into
>> >>> the
>> >>>>>>>>>>        changelog topic
>> >>>>>>>>>>        > >>>>>>>> and
>> >>>>>>>>>>        > >>>>>>>> thus, there are not conflicts.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Nevertheless, I don't see a big issue with
>> using
>> >>>>>>>>>>        headers within
>> >>>>>>>>>>        > >>>>>>>> Streams.
>> >>>>>>>>>>        > >>>>>>>> As long as we document it, we can have some
>> >>> "reserved"
>> >>>>>>>>>>        header keys
>> >>>>>>>>>>        > >>>>>>>> and
>> >>>>>>>>>>        > >>>>>>>> users are not allowed to use when processing
>> >>> data with
>> >>>>>>>>>>        Kafka
>> >>>>>>>>>>        > Streams.
>> >>>>>>>>>>        > >>>>>>>> IMHO, this should be ok.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> I think there is a safe way to avoid
>> conflicts,
>> >>> since
>> >>>>>>>>>> these
>> >>>>>>>>>>        > headers
>> >>>>>>>>>>        > >>>>>>>> are
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>> only needed in internal topics (I think):
>> >>>>>>>>>>        > >>>>>>>>> For internal and changelog topics, we can
>> >>> namespace
>> >>>>>>>>>>        all headers:
>> >>>>>>>>>>        > >>>>>>>>> * user-defined headers are namespaced as
>> >>> "external."
>> >>>>>>>>>> +
>> >>>>>>>>>>        headerKey
>> >>>>>>>>>>        > >>>>>>>>> * internal headers are namespaced as
>> >>> "internal." +
>> >>>>>>>>>>        headerKey
>> >>>>>>>>>>        > >>>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>> While name spacing would be possible, it
>> would
>> >>>>>>>>>> require
>> >>>>>>>>>>
>> >>>>>>>>> to
>> >>>>>>>
>> >>>>>>>>        > >>>>>>>> deserialize
>> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime
>> overhead. I
>> >>> would
>> >>>>>>>>>>        suggest to
>> >>>>>>>>>>        > no
>> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
>> this
>> >>>>>>>>>>
>> >>>>>>>>> becomes a
>> >>>>>>>
>> >>>>>>>>        > problem in
>> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
>> later
>> >>> on.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> My main concern about the design it the type
>> of
>> >>> the
>> >>>>>>>>>>        result KTable:
>> >>>>>>>>>>        > >>>>>>>> If I
>> >>>>>>>>>>        > >>>>>>>> understood the proposal correctly,
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> KTable<K1,V1> table1 = ...
>> >>>>>>>>>>        > >>>>>>>> KTable<K2,V2> table2 = ...
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> KTable<K1,V3> joinedTable =
>> >>> table1.join(table2,...);
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> implies that the `joinedTable` has the same
>> key
>> >>> as the
>> >>>>>>>>>>        left input
>> >>>>>>>>>>        > >>>>>>>> table.
>> >>>>>>>>>>        > >>>>>>>> IMHO, this does not work because if table2
>> >>> contains
>> >>>>>>>>>>        multiple rows
>> >>>>>>>>>>        > >>>>>>>> that
>> >>>>>>>>>>        > >>>>>>>> join with a record in table1 (what is the
>> main
>> >>> purpose
>> >>>>>>>>>>
>> >>>>>>>>> of
>> >>>>>>>
>> >>>>>>>> a
>> >>>>>>>>>>        > foreign
>> >>>>>>>>>>        > >>>>>>>> key
>> >>>>>>>>>>        > >>>>>>>> join), the result table would only contain a
>> >>> single
>> >>>>>>>>>>        join result,
>> >>>>>>>>>>        > but
>> >>>>>>>>>>        > >>>>>>>> not
>> >>>>>>>>>>        > >>>>>>>> multiple.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Example:
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> table1 input stream: <A,X>
>> >>>>>>>>>>        > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> We use table2 value a foreign key to table1
>> key
>> >>> (ie,
>> >>>>>>>>>>        "A" joins).
>> >>>>>>>>>>        > If
>> >>>>>>>>>>        > >>>>>>>> the
>> >>>>>>>>>>        > >>>>>>>> result key is the same key as key of table1,
>> this
>> >>>>>>>>>>        implies that the
>> >>>>>>>>>>        > >>>>>>>> result can either be <A, join(X,1)> or <A,
>> >>> join(X,2)>
>> >>>>>>>>>>        but not
>> >>>>>>>>>>        > both.
>> >>>>>>>>>>        > >>>>>>>> Because the share the same key, whatever
>> result
>> >>> record
>> >>>>>>>>>>        we emit
>> >>>>>>>>>>        > later,
>> >>>>>>>>>>        > >>>>>>>> overwrite the previous result.
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> This is the reason why Jan originally
>> proposed
>> >>> to use
>> >>>>>>>>>> a
>> >>>>>>>>>>        > combination
>> >>>>>>>>>>        > >>>>>>>> of
>> >>>>>>>>>>        > >>>>>>>> both primary keys of the input tables as key
>> of
>> >>> the
>> >>>>>>>>>>        output table.
>> >>>>>>>>>>        > >>>>>>>> This
>> >>>>>>>>>>        > >>>>>>>> makes the keys of the output table unique
>> and we
>> >>> can
>> >>>>>>>>>>        store both in
>> >>>>>>>>>>        > >>>>>>>> the
>> >>>>>>>>>>        > >>>>>>>> output table:
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>> >>> join(X,2)>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Thoughts?
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> -Matthias
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>> >>>>>>>>>>        > >>>>>>>>
>> >>>>>>>>>>        > >>>>>>>> Just on remark here.
>> >>>>>>>>>>        > >>>>>>>>> The high-watermark could be disregarded. The
>> >>> decision
>> >>>>>>>>>>        about the
>> >>>>>>>>>>        > >>>>>>>>> forward
>> >>>>>>>>>>        > >>>>>>>>> depends on the size of the aggregated map.
>> >>>>>>>>>>        > >>>>>>>>> Only 1 element long maps would be unpacked
>> and
>> >>>>>>>>>>        forwarded. 0
>> >>>>>>>>>>        > element
>> >>>>>>>>>>        > >>>>>>>>> maps
>> >>>>>>>>>>        > >>>>>>>>> would be published as delete. Any other
>> count
>> >>>>>>>>>>        > >>>>>>>>> of map entries is in "waiting for correct
>> >>> deletes to
>> >>>>>>>>>>        > arrive"-state.
>> >>>>>>>>>>        > >>>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>> >>>>>>>>>>        > >>>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>> It does look like I could replace the second
>> >>>>>>>>>>        repartition store
>> >>>>>>>>>>        > and
>> >>>>>>>>>>        > >>>>>>>>>> highwater store with a groupBy and reduce.
>> >>> However,
>> >>>>>>>>>>        it looks
>> >>>>>>>>>>        > like
>> >>>>>>>>>>        > >>>>>>>>>> I
>> >>>>>>>>>>        > >>>>>>>>>> would
>> >>>>>>>>>>        > >>>>>>>>>> still need to store the highwater value
>> within
>> >>> the
>> >>>>>>>>>>        materialized
>> >>>>>>>>>>        > >>>>>>>>>> store,
>> >>>>>>>>>>        > >>>>>>>>>>
>> >>>>>>>>>>        > >>>>>>>>>> to
>> >>>>>>>>>>        > >>>>>>>>> compare the arrival of out-of-order records
>> >>> (assuming
>> >>>>>>>>>>
>> >>>>>>>>> my
>> >>>>>>>
>> >>>>>>>>        > >>>>>>>>> understanding
>> >>>>>>>>>>        > >>>>>>>>> of
>> >>>>>>>>>>        > >>>>>>>>> THIS is correct...). This in effect is the
>> same
>> >>> as
>> >>>>>>>>>> the
>> >>>>>>>>>>        design I
>> >>>>>>>>>>        > have
>> >>>>>>>>>>        > >>>>>>>>> now,
>> >>>>>>>>>>        > >>>>>>>>> just with the two tables merged together.
>> >>>>>>>>>>        > >>>>>>>>>
>> >>>>>>>>>>        >
>> >>>>>>>>>>        >
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>> --
>> >>>>>>>> -- Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> --
>> >>>>>>> -- Guozhang
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
> I am more inclined to go with keeping it consistent and
>> separated into a normal repartition topic and a normal changelog topic
>> otherwise.

SGTM.


-Matthias

On 3/11/19 11:18 AM, Adam Bellemare wrote:
> Hi John
> 
> Thanks for the explanation. I wasn't sure how KTable repartition topics
> were handled with regards to cleanup but I just wanted to double check to
> see if it could cause an issue.
> 
> @Matthias
> My inclination is to keep the DSL topologies consistent with one another. I
> am a bit concerned about scope creep into the header domain, and I am not
> sure how much performance would be improved vs. additional complexity. I
> think if we go down this approach we should consider a new type of internal
> topic so that it's not confused with existing repartition and changelog
> topic types. I am more inclined to go with keeping it consistent and
> separated into a normal repartition topic and a normal changelog topic
> otherwise.
> 
> Thanks
> Adam
> 
> 
> 
> 
> 
> 
> On Mon, Mar 11, 2019 at 1:24 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> I guess Adam suggests, to use compaction for the repartition topic and
>> don't purge data. Doing this, would allow us to avoid a store changelog
>> topic for the "subscription store" on the RHS. This would be a nice
>> optimization.
>>
>> But the concern about breaking compaction is correct. However, I see it
>> as an optimization only and thus, if we keep the topic as plain
>> repartition topic and use a separate store changelog topic the issue
>> resolves itself.
>>
>> Maybe we could use headers thought to get this optimization. Do you
>> think it's worth to do this optimization or just stick with the simple
>> design and two topics (repartition plus changelog)?
>>
>>
>>
>> @Adam: thanks for updating the Wiki page. LGTM.
>>
>>
>> -Matthias
>>
>>
>> On 3/11/19 9:24 AM, John Roesler wrote:
>>> Hey Adam,
>>>
>>> That's a good observation, but it wouldn't be a problem for repartition
>>> topics because Streams aggressively deletes messages from the reparation
>>> topics once it knows they are handled. Thus, we don't need to try and
>> cater
>>> to the log compactor.
>>>
>>> Thanks,
>>> -John
>>>
>>> On Mon, Mar 11, 2019 at 9:10 AM Adam Bellemare <adam.bellemare@gmail.com
>>>
>>> wrote:
>>>
>>>> For the sake of expediency, I updated the KIP with what I believe we
>> have
>>>> discussed.
>>>>
>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges
>>>>
>>>>
>>>>
>>>> On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>>> wrote:
>>>>
>>>>> My only concern was around compaction of records in the repartition
>>>> topic.
>>>>> This would simply mean that these records would stick around as their
>>>> value
>>>>> isn't truly null. Though I know about the usage of compaction on
>>>> changelog
>>>>> topics, I am a bit fuzzy on where we use compaction in other internal
>>>>> topics. So long as this doesn't cause concern I can certainly finish
>> off
>>>>> the KIP today.
>>>>>
>>>>> Thanks
>>>>>
>>>>> Adam
>>>>>
>>>>> On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <
>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> I agree that the LHS side must encode this information and tell the
>> RHS
>>>>>> if a tombstone requires a reply or not.
>>>>>>
>>>>>>>> Would this pose some sort of verbosity problem in the internal
>>>> topics,
>>>>>>>> especially if we have to rebuild state off of them?
>>>>>>
>>>>>> I don't see an issue atm. Can you elaborate how this relates to
>> rebuild
>>>>>> state?
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 3/10/19 12:25 PM, Adam Bellemare wrote:
>>>>>>> Hi Matthias
>>>>>>>
>>>>>>> I have been mulling over the unsubscribe / delete optimization, and I
>>>>>> have
>>>>>>> one main concern. I believe that the RHS can only determine whether
>> to
>>>>>>> propagate the tombstone or not based on the value passed over from
>> the
>>>>>> LHS.
>>>>>>> This value would need to be non-null, and so wouldn't the internal
>>>>>>> repartition topics end up containing many non-null "tombstone"
>> values?
>>>>>>>
>>>>>>> ie:
>>>>>>> Normal tombstone (propagate):     (key=123, value=null)
>>>>>>> Don't-propagate-tombstone:          (key=123, value=("don't propagate
>>>>>> me,
>>>>>>> but please delete state"))
>>>>>>>
>>>>>>> Would this pose some sort of verbosity problem in the internal
>> topics,
>>>>>>> especially if we have to rebuild state off of them?
>>>>>>>
>>>>>>> Thanks
>>>>>>>
>>>>>>> Adam
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <
>>>> matthias@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> SGTM.
>>>>>>>>
>>>>>>>> I also had the impression that those duplicates are rather an error
>>>>>> than
>>>>>>>> an case of eventual consistency. Using hashing to avoid sending the
>>>>>>>> payload is a good idea IMHO.
>>>>>>>>
>>>>>>>> @Adam: can you update the KIP accordingly?
>>>>>>>>
>>>>>>>>  - add the optimization to not send a reply from RHS to LHS on
>>>>>>>> unsubscribe (if not a tombstone)
>>>>>>>>  - explain why using offsets to avoid duplicates does not work
>>>>>>>>  - add hashing to avoid duplicates
>>>>>>>>
>>>>>>>> Beside this, I don't have any further comments. Excited to finally
>>>> get
>>>>>>>> this in!
>>>>>>>>
>>>>>>>> Let us know when you have updated the KIP so we can move forward
>> with
>>>>>>>> the VOTE. Thanks a lot for your patience! This was a very loooong
>>>> shot!
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 3/8/19 8:47 AM, John Roesler wrote:
>>>>>>>>> Hi all,
>>>>>>>>>
>>>>>>>>> This proposal sounds good to me, especially since we observe that
>>>>>> people
>>>>>>>>> are already confused when the see duplicate results coming out of
>>>> 1:1
>>>>>>>> joins
>>>>>>>>> (which is a bug). I take this as "evidence" that we're better off
>>>>>>>>> eliminating those duplicates from the start. Guozhang's proposal
>>>> seems
>>>>>>>> like
>>>>>>>>> a lightweight solution to the problem, so FWIW, I'm in favor.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
>>>>>> adam.bellemare@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Guozhang
>>>>>>>>>>
>>>>>>>>>> That would certainly work for eliminating those duplicate values.
>>>> As
>>>>>> it
>>>>>>>>>> stands right now, this would be consistent with swallowing changes
>>>>>> due
>>>>>>>> to
>>>>>>>>>> out-of-order processing with multiple threads, and seems like a
>>>> very
>>>>>>>>>> reasonable way forward. Thank you for the suggestion!
>>>>>>>>>>
>>>>>>>>>> I have been trying to think if there are any other scenarios where
>>>> we
>>>>>>>> can
>>>>>>>>>> end up with duplicates, though I have not been able to identify
>> any
>>>>>>>> others
>>>>>>>>>> at the moment. I will think on it a bit more, but if anyone else
>>>> has
>>>>>> any
>>>>>>>>>> ideas, please chime in.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Adam
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
>>>>>>>> separate
>>>>>>>>>>> this issue with the more general issue that today (not only
>>>>>>>> foreign-key,
>>>>>>>>>>> but also co-partition primary-key) table-table joins is still not
>>>>>>>>>> strictly
>>>>>>>>>>> respecting the timestamp ordering since the two changelog streams
>>>>>> may
>>>>>>>> be
>>>>>>>>>>> fetched and hence processed out-of-order and we do not allow a
>>>>>> record
>>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>> joined with the other table at any given time snapshot yet. So
>>>>>> ideally
>>>>>>>>>> when
>>>>>>>>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1,
>> v2))
>>>>>>>> coming
>>>>>>>>>>> at the left hand table and one record (f-k1, v3) at the right
>> hand
>>>>>>>> table,
>>>>>>>>>>> depending on the processing ordering we may get:
>>>>>>>>>>>
>>>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>>>
>>>>>>>>>>> or
>>>>>>>>>>>
>>>>>>>>>>> (k1, (f-k1, v1-v3))
>>>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>>>
>>>>>>>>>>> And this is not to be addressed by this KIP.
>>>>>>>>>>>
>>>>>>>>>>> What I would advocate is to fix the issue that is introduced in
>>>> this
>>>>>>>> KIP
>>>>>>>>>>> alone, that is we may have
>>>>>>>>>>>
>>>>>>>>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
>>>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>>>
>>>>>>>>>>> I admit that it does not have correctness issue from the
>> semantics
>>>>>>>> along,
>>>>>>>>>>> comparing it with "discarding the first result", but it may be
>>>>>>>> confusing
>>>>>>>>>>> from user's observation who do not expect to see the seemingly
>>>>>>>>>> duplicates.
>>>>>>>>>>> On the other hand, I think there's a light solution to avoid it,
>>>>>> which
>>>>>>>> is
>>>>>>>>>>> that we can still optimize away to not send the full payload of
>>>> "v1"
>>>>>>>> from
>>>>>>>>>>> left hand side to right hand side, but instead of just trimming
>>>> off
>>>>>> the
>>>>>>>>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm
>>>> using
>>>>>> MD5
>>>>>>>>>>> here just as an example, we can definitely replace it with other
>>>>>>>>>>> functions), by doing which we can discard the join operation if
>>>> the
>>>>>>>> hash
>>>>>>>>>>> value sent back from the right hand side does not match with the
>>>>>> left
>>>>>>>>>> hand
>>>>>>>>>>> side any more, i.e. we will only send:
>>>>>>>>>>>
>>>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>>>
>>>>>>>>>>> to down streams once.
>>>>>>>>>>>
>>>>>>>>>>> WDYT?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Ah yes, I recall it all now. That answers that question as to
>>>> why I
>>>>>>>> had
>>>>>>>>>>>> caching disabled. I can certainly re-enable it since I believe
>>>> the
>>>>>>>> main
>>>>>>>>>>>> concern was simply about reconciling those two iterators. A lack
>>>> of
>>>>>>>>>>>> knowledge there on my part.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thank you John for weighing in - we certainly both do appreciate
>>>>>> it. I
>>>>>>>>>>>> think that John hits it on the head though with his comment of
>>>> "If
>>>>>> it
>>>>>>>>>>> turns
>>>>>>>>>>>> out we're wrong about this, then it should be possible to fix
>> the
>>>>>>>>>>> semantics
>>>>>>>>>>>> in place, without messing with the API."
>>>>>>>>>>>>
>>>>>>>>>>>> If anyone else would like to weigh in, your thoughts would be
>>>>>> greatly
>>>>>>>>>>>> appreciated.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
>>>>>> matthias@confluent.io
>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>>> I dont know how to range scan over a caching store, probably
>>>> one
>>>>>>>>>> had
>>>>>>>>>>>>>>> to open 2 iterators and merge them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That happens automatically. If you query a cached KTable, it
>>>>>> ranges
>>>>>>>>>>> over
>>>>>>>>>>>>> the cache and the underlying RocksDB and performs the merging
>>>>>> under
>>>>>>>>>> the
>>>>>>>>>>>>> hood.
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Other than that, I still think even the regualr join is
>> broken
>>>>>>>>>> with
>>>>>>>>>>>>>>> caching enabled right?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Why? To me, if you use the word "broker", it implies
>>>> conceptually
>>>>>>>>>>>>> incorrect; I don't see this.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I once files a ticket, because with caching
>>>>>>>>>>>>>>>> enabled it would return values that havent been published
>>>>>>>>>>> downstream
>>>>>>>>>>>>> yet.
>>>>>>>>>>>>>
>>>>>>>>>>>>> For the bug report, if found
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still
>> need
>>>>>> to
>>>>>>>>>> fix
>>>>>>>>>>>>> this, but it is a regular bug as any other, and we should not
>>>>>> change
>>>>>>>>>> a
>>>>>>>>>>>>> design because of a bug.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That range() returns values that have not been published
>>>>>> downstream
>>>>>>>>>> if
>>>>>>>>>>>>> caching is enabled is how caching works and is intended
>>>> behavior.
>>>>>> Not
>>>>>>>>>>>>> sure why say it's incorrect?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
>>>>>>>>>>>>>>> Thanks Adam,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there
>>>> is
>>>>>>>>>> no
>>>>>>>>>>>>>>> functional/correctness requirement to disable caching. I
>>>> cannot
>>>>>>>>>>>>>>> remember why Jan's proposal added this? It might be an
>>>>>>>>>>>>>>> implementation detail though (maybe just remove it from the
>>>> KIP?
>>>>>>>>>>>>>>> -- might be miss leading).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I dont know how to range scan over a caching store, probably
>>>> one
>>>>>>>>>> had
>>>>>>>>>>>>>> to open 2 iterators and merge them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Other than that, I still think even the regualr join is broken
>>>>>> with
>>>>>>>>>>>>>> caching enabled right? I once files a ticket, because with
>>>>>> caching
>>>>>>>>>>>>>> enabled it would return values that havent been published
>>>>>>>>>> downstream
>>>>>>>>>>>> yet.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>>
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Guozhang Wang <wa...@gmail.com>.
I'd agree with Adam as well: we could consider optimizations leveraging
headers as a separate topic that applies more broadly in the future.

On Mon, Mar 11, 2019 at 11:29 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi John
>
> Thanks for the explanation. I wasn't sure how KTable repartition topics
> were handled with regards to cleanup but I just wanted to double check to
> see if it could cause an issue.
>
> @Matthias
> My inclination is to keep the DSL topologies consistent with one another. I
> am a bit concerned about scope creep into the header domain, and I am not
> sure how much performance would be improved vs. additional complexity. I
> think if we go down this approach we should consider a new type of internal
> topic so that it's not confused with existing repartition and changelog
> topic types. I am more inclined to go with keeping it consistent and
> separated into a normal repartition topic and a normal changelog topic
> otherwise.
>
> Thanks
> Adam
>
>
>
>
>
>
> On Mon, Mar 11, 2019 at 1:24 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > I guess Adam suggests, to use compaction for the repartition topic and
> > don't purge data. Doing this, would allow us to avoid a store changelog
> > topic for the "subscription store" on the RHS. This would be a nice
> > optimization.
> >
> > But the concern about breaking compaction is correct. However, I see it
> > as an optimization only and thus, if we keep the topic as plain
> > repartition topic and use a separate store changelog topic the issue
> > resolves itself.
> >
> > Maybe we could use headers thought to get this optimization. Do you
> > think it's worth to do this optimization or just stick with the simple
> > design and two topics (repartition plus changelog)?
> >
> >
> >
> > @Adam: thanks for updating the Wiki page. LGTM.
> >
> >
> > -Matthias
> >
> >
> > On 3/11/19 9:24 AM, John Roesler wrote:
> > > Hey Adam,
> > >
> > > That's a good observation, but it wouldn't be a problem for repartition
> > > topics because Streams aggressively deletes messages from the
> reparation
> > > topics once it knows they are handled. Thus, we don't need to try and
> > cater
> > > to the log compactor.
> > >
> > > Thanks,
> > > -John
> > >
> > > On Mon, Mar 11, 2019 at 9:10 AM Adam Bellemare <
> adam.bellemare@gmail.com
> > >
> > > wrote:
> > >
> > >> For the sake of expediency, I updated the KIP with what I believe we
> > have
> > >> discussed.
> > >>
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges
> > >>
> > >>
> > >>
> > >> On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <
> > adam.bellemare@gmail.com>
> > >> wrote:
> > >>
> > >>> My only concern was around compaction of records in the repartition
> > >> topic.
> > >>> This would simply mean that these records would stick around as their
> > >> value
> > >>> isn't truly null. Though I know about the usage of compaction on
> > >> changelog
> > >>> topics, I am a bit fuzzy on where we use compaction in other internal
> > >>> topics. So long as this doesn't cause concern I can certainly finish
> > off
> > >>> the KIP today.
> > >>>
> > >>> Thanks
> > >>>
> > >>> Adam
> > >>>
> > >>> On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <
> > matthias@confluent.io>
> > >>> wrote:
> > >>>
> > >>>> I agree that the LHS side must encode this information and tell the
> > RHS
> > >>>> if a tombstone requires a reply or not.
> > >>>>
> > >>>>>> Would this pose some sort of verbosity problem in the internal
> > >> topics,
> > >>>>>> especially if we have to rebuild state off of them?
> > >>>>
> > >>>> I don't see an issue atm. Can you elaborate how this relates to
> > rebuild
> > >>>> state?
> > >>>>
> > >>>>
> > >>>> -Matthias
> > >>>>
> > >>>> On 3/10/19 12:25 PM, Adam Bellemare wrote:
> > >>>>> Hi Matthias
> > >>>>>
> > >>>>> I have been mulling over the unsubscribe / delete optimization,
> and I
> > >>>> have
> > >>>>> one main concern. I believe that the RHS can only determine whether
> > to
> > >>>>> propagate the tombstone or not based on the value passed over from
> > the
> > >>>> LHS.
> > >>>>> This value would need to be non-null, and so wouldn't the internal
> > >>>>> repartition topics end up containing many non-null "tombstone"
> > values?
> > >>>>>
> > >>>>> ie:
> > >>>>> Normal tombstone (propagate):     (key=123, value=null)
> > >>>>> Don't-propagate-tombstone:          (key=123, value=("don't
> propagate
> > >>>> me,
> > >>>>> but please delete state"))
> > >>>>>
> > >>>>> Would this pose some sort of verbosity problem in the internal
> > topics,
> > >>>>> especially if we have to rebuild state off of them?
> > >>>>>
> > >>>>> Thanks
> > >>>>>
> > >>>>> Adam
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <
> > >> matthias@confluent.io>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> SGTM.
> > >>>>>>
> > >>>>>> I also had the impression that those duplicates are rather an
> error
> > >>>> than
> > >>>>>> an case of eventual consistency. Using hashing to avoid sending
> the
> > >>>>>> payload is a good idea IMHO.
> > >>>>>>
> > >>>>>> @Adam: can you update the KIP accordingly?
> > >>>>>>
> > >>>>>>  - add the optimization to not send a reply from RHS to LHS on
> > >>>>>> unsubscribe (if not a tombstone)
> > >>>>>>  - explain why using offsets to avoid duplicates does not work
> > >>>>>>  - add hashing to avoid duplicates
> > >>>>>>
> > >>>>>> Beside this, I don't have any further comments. Excited to finally
> > >> get
> > >>>>>> this in!
> > >>>>>>
> > >>>>>> Let us know when you have updated the KIP so we can move forward
> > with
> > >>>>>> the VOTE. Thanks a lot for your patience! This was a very loooong
> > >> shot!
> > >>>>>>
> > >>>>>>
> > >>>>>> -Matthias
> > >>>>>>
> > >>>>>> On 3/8/19 8:47 AM, John Roesler wrote:
> > >>>>>>> Hi all,
> > >>>>>>>
> > >>>>>>> This proposal sounds good to me, especially since we observe that
> > >>>> people
> > >>>>>>> are already confused when the see duplicate results coming out of
> > >> 1:1
> > >>>>>> joins
> > >>>>>>> (which is a bug). I take this as "evidence" that we're better off
> > >>>>>>> eliminating those duplicates from the start. Guozhang's proposal
> > >> seems
> > >>>>>> like
> > >>>>>>> a lightweight solution to the problem, so FWIW, I'm in favor.
> > >>>>>>>
> > >>>>>>> Thanks,
> > >>>>>>> -John
> > >>>>>>>
> > >>>>>>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
> > >>>> adam.bellemare@gmail.com>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> Hi Guozhang
> > >>>>>>>>
> > >>>>>>>> That would certainly work for eliminating those duplicate
> values.
> > >> As
> > >>>> it
> > >>>>>>>> stands right now, this would be consistent with swallowing
> changes
> > >>>> due
> > >>>>>> to
> > >>>>>>>> out-of-order processing with multiple threads, and seems like a
> > >> very
> > >>>>>>>> reasonable way forward. Thank you for the suggestion!
> > >>>>>>>>
> > >>>>>>>> I have been trying to think if there are any other scenarios
> where
> > >> we
> > >>>>>> can
> > >>>>>>>> end up with duplicates, though I have not been able to identify
> > any
> > >>>>>> others
> > >>>>>>>> at the moment. I will think on it a bit more, but if anyone else
> > >> has
> > >>>> any
> > >>>>>>>> ideas, please chime in.
> > >>>>>>>>
> > >>>>>>>> Thanks,
> > >>>>>>>> Adam
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <
> wangguoz@gmail.com>
> > >>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
> > >>>>>> separate
> > >>>>>>>>> this issue with the more general issue that today (not only
> > >>>>>> foreign-key,
> > >>>>>>>>> but also co-partition primary-key) table-table joins is still
> not
> > >>>>>>>> strictly
> > >>>>>>>>> respecting the timestamp ordering since the two changelog
> streams
> > >>>> may
> > >>>>>> be
> > >>>>>>>>> fetched and hence processed out-of-order and we do not allow a
> > >>>> record
> > >>>>>> to
> > >>>>>>>> be
> > >>>>>>>>> joined with the other table at any given time snapshot yet. So
> > >>>> ideally
> > >>>>>>>> when
> > >>>>>>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1,
> > v2))
> > >>>>>> coming
> > >>>>>>>>> at the left hand table and one record (f-k1, v3) at the right
> > hand
> > >>>>>> table,
> > >>>>>>>>> depending on the processing ordering we may get:
> > >>>>>>>>>
> > >>>>>>>>> (k1, (f-k1, v2-v3))
> > >>>>>>>>>
> > >>>>>>>>> or
> > >>>>>>>>>
> > >>>>>>>>> (k1, (f-k1, v1-v3))
> > >>>>>>>>> (k1, (f-k1, v2-v3))
> > >>>>>>>>>
> > >>>>>>>>> And this is not to be addressed by this KIP.
> > >>>>>>>>>
> > >>>>>>>>> What I would advocate is to fix the issue that is introduced in
> > >> this
> > >>>>>> KIP
> > >>>>>>>>> alone, that is we may have
> > >>>>>>>>>
> > >>>>>>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> > >>>>>>>>> (k1, (f-k1, v2-v3))
> > >>>>>>>>>
> > >>>>>>>>> I admit that it does not have correctness issue from the
> > semantics
> > >>>>>> along,
> > >>>>>>>>> comparing it with "discarding the first result", but it may be
> > >>>>>> confusing
> > >>>>>>>>> from user's observation who do not expect to see the seemingly
> > >>>>>>>> duplicates.
> > >>>>>>>>> On the other hand, I think there's a light solution to avoid
> it,
> > >>>> which
> > >>>>>> is
> > >>>>>>>>> that we can still optimize away to not send the full payload of
> > >> "v1"
> > >>>>>> from
> > >>>>>>>>> left hand side to right hand side, but instead of just trimming
> > >> off
> > >>>> the
> > >>>>>>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm
> > >> using
> > >>>> MD5
> > >>>>>>>>> here just as an example, we can definitely replace it with
> other
> > >>>>>>>>> functions), by doing which we can discard the join operation if
> > >> the
> > >>>>>> hash
> > >>>>>>>>> value sent back from the right hand side does not match with
> the
> > >>>> left
> > >>>>>>>> hand
> > >>>>>>>>> side any more, i.e. we will only send:
> > >>>>>>>>>
> > >>>>>>>>> (k1, (f-k1, v2-v3))
> > >>>>>>>>>
> > >>>>>>>>> to down streams once.
> > >>>>>>>>>
> > >>>>>>>>> WDYT?
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> Guozhang
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
> > >>>>>> adam.bellemare@gmail.com>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Ah yes, I recall it all now. That answers that question as to
> > >> why I
> > >>>>>> had
> > >>>>>>>>>> caching disabled. I can certainly re-enable it since I believe
> > >> the
> > >>>>>> main
> > >>>>>>>>>> concern was simply about reconciling those two iterators. A
> lack
> > >> of
> > >>>>>>>>>> knowledge there on my part.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Thank you John for weighing in - we certainly both do
> appreciate
> > >>>> it. I
> > >>>>>>>>>> think that John hits it on the head though with his comment of
> > >> "If
> > >>>> it
> > >>>>>>>>> turns
> > >>>>>>>>>> out we're wrong about this, then it should be possible to fix
> > the
> > >>>>>>>>> semantics
> > >>>>>>>>>> in place, without messing with the API."
> > >>>>>>>>>>
> > >>>>>>>>>> If anyone else would like to weigh in, your thoughts would be
> > >>>> greatly
> > >>>>>>>>>> appreciated.
> > >>>>>>>>>>
> > >>>>>>>>>> Thanks
> > >>>>>>>>>>
> > >>>>>>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
> > >>>> matthias@confluent.io
> > >>>>>>>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>>>> I dont know how to range scan over a caching store,
> probably
> > >> one
> > >>>>>>>> had
> > >>>>>>>>>>>>> to open 2 iterators and merge them.
> > >>>>>>>>>>>
> > >>>>>>>>>>> That happens automatically. If you query a cached KTable, it
> > >>>> ranges
> > >>>>>>>>> over
> > >>>>>>>>>>> the cache and the underlying RocksDB and performs the merging
> > >>>> under
> > >>>>>>>> the
> > >>>>>>>>>>> hood.
> > >>>>>>>>>>>
> > >>>>>>>>>>>>> Other than that, I still think even the regualr join is
> > broken
> > >>>>>>>> with
> > >>>>>>>>>>>>> caching enabled right?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Why? To me, if you use the word "broker", it implies
> > >> conceptually
> > >>>>>>>>>>> incorrect; I don't see this.
> > >>>>>>>>>>>
> > >>>>>>>>>>>> I once files a ticket, because with caching
> > >>>>>>>>>>>>>> enabled it would return values that havent been published
> > >>>>>>>>> downstream
> > >>>>>>>>>>> yet.
> > >>>>>>>>>>>
> > >>>>>>>>>>> For the bug report, if found
> > >>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still
> > need
> > >>>> to
> > >>>>>>>> fix
> > >>>>>>>>>>> this, but it is a regular bug as any other, and we should not
> > >>>> change
> > >>>>>>>> a
> > >>>>>>>>>>> design because of a bug.
> > >>>>>>>>>>>
> > >>>>>>>>>>> That range() returns values that have not been published
> > >>>> downstream
> > >>>>>>>> if
> > >>>>>>>>>>> caching is enabled is how caching works and is intended
> > >> behavior.
> > >>>> Not
> > >>>>>>>>>>> sure why say it's incorrect?
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> -Matthias
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
> > >>>>>>>>>>>>> Thanks Adam,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus,
> there
> > >> is
> > >>>>>>>> no
> > >>>>>>>>>>>>> functional/correctness requirement to disable caching. I
> > >> cannot
> > >>>>>>>>>>>>> remember why Jan's proposal added this? It might be an
> > >>>>>>>>>>>>> implementation detail though (maybe just remove it from the
> > >> KIP?
> > >>>>>>>>>>>>> -- might be miss leading).
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I dont know how to range scan over a caching store, probably
> > >> one
> > >>>>>>>> had
> > >>>>>>>>>>>> to open 2 iterators and merge them.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Other than that, I still think even the regualr join is
> broken
> > >>>> with
> > >>>>>>>>>>>> caching enabled right? I once files a ticket, because with
> > >>>> caching
> > >>>>>>>>>>>> enabled it would return values that havent been published
> > >>>>>>>> downstream
> > >>>>>>>>>> yet.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> --
> > >>>>>>>>> -- Guozhang
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>
> > >
> >
> >
>


-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi John

Thanks for the explanation. I wasn't sure how KTable repartition topics
were handled with regards to cleanup but I just wanted to double check to
see if it could cause an issue.

@Matthias
My inclination is to keep the DSL topologies consistent with one another. I
am a bit concerned about scope creep into the header domain, and I am not
sure how much performance would be improved vs. additional complexity. I
think if we go down this approach we should consider a new type of internal
topic so that it's not confused with existing repartition and changelog
topic types. I am more inclined to go with keeping it consistent and
separated into a normal repartition topic and a normal changelog topic
otherwise.

Thanks
Adam






On Mon, Mar 11, 2019 at 1:24 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> I guess Adam suggests, to use compaction for the repartition topic and
> don't purge data. Doing this, would allow us to avoid a store changelog
> topic for the "subscription store" on the RHS. This would be a nice
> optimization.
>
> But the concern about breaking compaction is correct. However, I see it
> as an optimization only and thus, if we keep the topic as plain
> repartition topic and use a separate store changelog topic the issue
> resolves itself.
>
> Maybe we could use headers thought to get this optimization. Do you
> think it's worth to do this optimization or just stick with the simple
> design and two topics (repartition plus changelog)?
>
>
>
> @Adam: thanks for updating the Wiki page. LGTM.
>
>
> -Matthias
>
>
> On 3/11/19 9:24 AM, John Roesler wrote:
> > Hey Adam,
> >
> > That's a good observation, but it wouldn't be a problem for repartition
> > topics because Streams aggressively deletes messages from the reparation
> > topics once it knows they are handled. Thus, we don't need to try and
> cater
> > to the log compactor.
> >
> > Thanks,
> > -John
> >
> > On Mon, Mar 11, 2019 at 9:10 AM Adam Bellemare <adam.bellemare@gmail.com
> >
> > wrote:
> >
> >> For the sake of expediency, I updated the KIP with what I believe we
> have
> >> discussed.
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges
> >>
> >>
> >>
> >> On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <
> adam.bellemare@gmail.com>
> >> wrote:
> >>
> >>> My only concern was around compaction of records in the repartition
> >> topic.
> >>> This would simply mean that these records would stick around as their
> >> value
> >>> isn't truly null. Though I know about the usage of compaction on
> >> changelog
> >>> topics, I am a bit fuzzy on where we use compaction in other internal
> >>> topics. So long as this doesn't cause concern I can certainly finish
> off
> >>> the KIP today.
> >>>
> >>> Thanks
> >>>
> >>> Adam
> >>>
> >>> On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <
> matthias@confluent.io>
> >>> wrote:
> >>>
> >>>> I agree that the LHS side must encode this information and tell the
> RHS
> >>>> if a tombstone requires a reply or not.
> >>>>
> >>>>>> Would this pose some sort of verbosity problem in the internal
> >> topics,
> >>>>>> especially if we have to rebuild state off of them?
> >>>>
> >>>> I don't see an issue atm. Can you elaborate how this relates to
> rebuild
> >>>> state?
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 3/10/19 12:25 PM, Adam Bellemare wrote:
> >>>>> Hi Matthias
> >>>>>
> >>>>> I have been mulling over the unsubscribe / delete optimization, and I
> >>>> have
> >>>>> one main concern. I believe that the RHS can only determine whether
> to
> >>>>> propagate the tombstone or not based on the value passed over from
> the
> >>>> LHS.
> >>>>> This value would need to be non-null, and so wouldn't the internal
> >>>>> repartition topics end up containing many non-null "tombstone"
> values?
> >>>>>
> >>>>> ie:
> >>>>> Normal tombstone (propagate):     (key=123, value=null)
> >>>>> Don't-propagate-tombstone:          (key=123, value=("don't propagate
> >>>> me,
> >>>>> but please delete state"))
> >>>>>
> >>>>> Would this pose some sort of verbosity problem in the internal
> topics,
> >>>>> especially if we have to rebuild state off of them?
> >>>>>
> >>>>> Thanks
> >>>>>
> >>>>> Adam
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <
> >> matthias@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> SGTM.
> >>>>>>
> >>>>>> I also had the impression that those duplicates are rather an error
> >>>> than
> >>>>>> an case of eventual consistency. Using hashing to avoid sending the
> >>>>>> payload is a good idea IMHO.
> >>>>>>
> >>>>>> @Adam: can you update the KIP accordingly?
> >>>>>>
> >>>>>>  - add the optimization to not send a reply from RHS to LHS on
> >>>>>> unsubscribe (if not a tombstone)
> >>>>>>  - explain why using offsets to avoid duplicates does not work
> >>>>>>  - add hashing to avoid duplicates
> >>>>>>
> >>>>>> Beside this, I don't have any further comments. Excited to finally
> >> get
> >>>>>> this in!
> >>>>>>
> >>>>>> Let us know when you have updated the KIP so we can move forward
> with
> >>>>>> the VOTE. Thanks a lot for your patience! This was a very loooong
> >> shot!
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 3/8/19 8:47 AM, John Roesler wrote:
> >>>>>>> Hi all,
> >>>>>>>
> >>>>>>> This proposal sounds good to me, especially since we observe that
> >>>> people
> >>>>>>> are already confused when the see duplicate results coming out of
> >> 1:1
> >>>>>> joins
> >>>>>>> (which is a bug). I take this as "evidence" that we're better off
> >>>>>>> eliminating those duplicates from the start. Guozhang's proposal
> >> seems
> >>>>>> like
> >>>>>>> a lightweight solution to the problem, so FWIW, I'm in favor.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> -John
> >>>>>>>
> >>>>>>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
> >>>> adam.bellemare@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Guozhang
> >>>>>>>>
> >>>>>>>> That would certainly work for eliminating those duplicate values.
> >> As
> >>>> it
> >>>>>>>> stands right now, this would be consistent with swallowing changes
> >>>> due
> >>>>>> to
> >>>>>>>> out-of-order processing with multiple threads, and seems like a
> >> very
> >>>>>>>> reasonable way forward. Thank you for the suggestion!
> >>>>>>>>
> >>>>>>>> I have been trying to think if there are any other scenarios where
> >> we
> >>>>>> can
> >>>>>>>> end up with duplicates, though I have not been able to identify
> any
> >>>>>> others
> >>>>>>>> at the moment. I will think on it a bit more, but if anyone else
> >> has
> >>>> any
> >>>>>>>> ideas, please chime in.
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> Adam
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
> >>>>>> separate
> >>>>>>>>> this issue with the more general issue that today (not only
> >>>>>> foreign-key,
> >>>>>>>>> but also co-partition primary-key) table-table joins is still not
> >>>>>>>> strictly
> >>>>>>>>> respecting the timestamp ordering since the two changelog streams
> >>>> may
> >>>>>> be
> >>>>>>>>> fetched and hence processed out-of-order and we do not allow a
> >>>> record
> >>>>>> to
> >>>>>>>> be
> >>>>>>>>> joined with the other table at any given time snapshot yet. So
> >>>> ideally
> >>>>>>>> when
> >>>>>>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1,
> v2))
> >>>>>> coming
> >>>>>>>>> at the left hand table and one record (f-k1, v3) at the right
> hand
> >>>>>> table,
> >>>>>>>>> depending on the processing ordering we may get:
> >>>>>>>>>
> >>>>>>>>> (k1, (f-k1, v2-v3))
> >>>>>>>>>
> >>>>>>>>> or
> >>>>>>>>>
> >>>>>>>>> (k1, (f-k1, v1-v3))
> >>>>>>>>> (k1, (f-k1, v2-v3))
> >>>>>>>>>
> >>>>>>>>> And this is not to be addressed by this KIP.
> >>>>>>>>>
> >>>>>>>>> What I would advocate is to fix the issue that is introduced in
> >> this
> >>>>>> KIP
> >>>>>>>>> alone, that is we may have
> >>>>>>>>>
> >>>>>>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> >>>>>>>>> (k1, (f-k1, v2-v3))
> >>>>>>>>>
> >>>>>>>>> I admit that it does not have correctness issue from the
> semantics
> >>>>>> along,
> >>>>>>>>> comparing it with "discarding the first result", but it may be
> >>>>>> confusing
> >>>>>>>>> from user's observation who do not expect to see the seemingly
> >>>>>>>> duplicates.
> >>>>>>>>> On the other hand, I think there's a light solution to avoid it,
> >>>> which
> >>>>>> is
> >>>>>>>>> that we can still optimize away to not send the full payload of
> >> "v1"
> >>>>>> from
> >>>>>>>>> left hand side to right hand side, but instead of just trimming
> >> off
> >>>> the
> >>>>>>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm
> >> using
> >>>> MD5
> >>>>>>>>> here just as an example, we can definitely replace it with other
> >>>>>>>>> functions), by doing which we can discard the join operation if
> >> the
> >>>>>> hash
> >>>>>>>>> value sent back from the right hand side does not match with the
> >>>> left
> >>>>>>>> hand
> >>>>>>>>> side any more, i.e. we will only send:
> >>>>>>>>>
> >>>>>>>>> (k1, (f-k1, v2-v3))
> >>>>>>>>>
> >>>>>>>>> to down streams once.
> >>>>>>>>>
> >>>>>>>>> WDYT?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
> >>>>>> adam.bellemare@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Ah yes, I recall it all now. That answers that question as to
> >> why I
> >>>>>> had
> >>>>>>>>>> caching disabled. I can certainly re-enable it since I believe
> >> the
> >>>>>> main
> >>>>>>>>>> concern was simply about reconciling those two iterators. A lack
> >> of
> >>>>>>>>>> knowledge there on my part.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thank you John for weighing in - we certainly both do appreciate
> >>>> it. I
> >>>>>>>>>> think that John hits it on the head though with his comment of
> >> "If
> >>>> it
> >>>>>>>>> turns
> >>>>>>>>>> out we're wrong about this, then it should be possible to fix
> the
> >>>>>>>>> semantics
> >>>>>>>>>> in place, without messing with the API."
> >>>>>>>>>>
> >>>>>>>>>> If anyone else would like to weigh in, your thoughts would be
> >>>> greatly
> >>>>>>>>>> appreciated.
> >>>>>>>>>>
> >>>>>>>>>> Thanks
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
> >>>> matthias@confluent.io
> >>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>>>> I dont know how to range scan over a caching store, probably
> >> one
> >>>>>>>> had
> >>>>>>>>>>>>> to open 2 iterators and merge them.
> >>>>>>>>>>>
> >>>>>>>>>>> That happens automatically. If you query a cached KTable, it
> >>>> ranges
> >>>>>>>>> over
> >>>>>>>>>>> the cache and the underlying RocksDB and performs the merging
> >>>> under
> >>>>>>>> the
> >>>>>>>>>>> hood.
> >>>>>>>>>>>
> >>>>>>>>>>>>> Other than that, I still think even the regualr join is
> broken
> >>>>>>>> with
> >>>>>>>>>>>>> caching enabled right?
> >>>>>>>>>>>
> >>>>>>>>>>> Why? To me, if you use the word "broker", it implies
> >> conceptually
> >>>>>>>>>>> incorrect; I don't see this.
> >>>>>>>>>>>
> >>>>>>>>>>>> I once files a ticket, because with caching
> >>>>>>>>>>>>>> enabled it would return values that havent been published
> >>>>>>>>> downstream
> >>>>>>>>>>> yet.
> >>>>>>>>>>>
> >>>>>>>>>>> For the bug report, if found
> >>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still
> need
> >>>> to
> >>>>>>>> fix
> >>>>>>>>>>> this, but it is a regular bug as any other, and we should not
> >>>> change
> >>>>>>>> a
> >>>>>>>>>>> design because of a bug.
> >>>>>>>>>>>
> >>>>>>>>>>> That range() returns values that have not been published
> >>>> downstream
> >>>>>>>> if
> >>>>>>>>>>> caching is enabled is how caching works and is intended
> >> behavior.
> >>>> Not
> >>>>>>>>>>> sure why say it's incorrect?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
> >>>>>>>>>>>>> Thanks Adam,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there
> >> is
> >>>>>>>> no
> >>>>>>>>>>>>> functional/correctness requirement to disable caching. I
> >> cannot
> >>>>>>>>>>>>> remember why Jan's proposal added this? It might be an
> >>>>>>>>>>>>> implementation detail though (maybe just remove it from the
> >> KIP?
> >>>>>>>>>>>>> -- might be miss leading).
> >>>>>>>>>>>>
> >>>>>>>>>>>> I dont know how to range scan over a caching store, probably
> >> one
> >>>>>>>> had
> >>>>>>>>>>>> to open 2 iterators and merge them.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Other than that, I still think even the regualr join is broken
> >>>> with
> >>>>>>>>>>>> caching enabled right? I once files a ticket, because with
> >>>> caching
> >>>>>>>>>>>> enabled it would return values that havent been published
> >>>>>>>> downstream
> >>>>>>>>>> yet.
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> --
> >>>>>>>>> -- Guozhang
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I guess Adam suggests, to use compaction for the repartition topic and
don't purge data. Doing this, would allow us to avoid a store changelog
topic for the "subscription store" on the RHS. This would be a nice
optimization.

But the concern about breaking compaction is correct. However, I see it
as an optimization only and thus, if we keep the topic as plain
repartition topic and use a separate store changelog topic the issue
resolves itself.

Maybe we could use headers thought to get this optimization. Do you
think it's worth to do this optimization or just stick with the simple
design and two topics (repartition plus changelog)?



@Adam: thanks for updating the Wiki page. LGTM.


-Matthias


On 3/11/19 9:24 AM, John Roesler wrote:
> Hey Adam,
> 
> That's a good observation, but it wouldn't be a problem for repartition
> topics because Streams aggressively deletes messages from the reparation
> topics once it knows they are handled. Thus, we don't need to try and cater
> to the log compactor.
> 
> Thanks,
> -John
> 
> On Mon, Mar 11, 2019 at 9:10 AM Adam Bellemare <ad...@gmail.com>
> wrote:
> 
>> For the sake of expediency, I updated the KIP with what I believe we have
>> discussed.
>>
>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges
>>
>>
>>
>> On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <ad...@gmail.com>
>> wrote:
>>
>>> My only concern was around compaction of records in the repartition
>> topic.
>>> This would simply mean that these records would stick around as their
>> value
>>> isn't truly null. Though I know about the usage of compaction on
>> changelog
>>> topics, I am a bit fuzzy on where we use compaction in other internal
>>> topics. So long as this doesn't cause concern I can certainly finish off
>>> the KIP today.
>>>
>>> Thanks
>>>
>>> Adam
>>>
>>> On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> I agree that the LHS side must encode this information and tell the RHS
>>>> if a tombstone requires a reply or not.
>>>>
>>>>>> Would this pose some sort of verbosity problem in the internal
>> topics,
>>>>>> especially if we have to rebuild state off of them?
>>>>
>>>> I don't see an issue atm. Can you elaborate how this relates to rebuild
>>>> state?
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 3/10/19 12:25 PM, Adam Bellemare wrote:
>>>>> Hi Matthias
>>>>>
>>>>> I have been mulling over the unsubscribe / delete optimization, and I
>>>> have
>>>>> one main concern. I believe that the RHS can only determine whether to
>>>>> propagate the tombstone or not based on the value passed over from the
>>>> LHS.
>>>>> This value would need to be non-null, and so wouldn't the internal
>>>>> repartition topics end up containing many non-null "tombstone" values?
>>>>>
>>>>> ie:
>>>>> Normal tombstone (propagate):     (key=123, value=null)
>>>>> Don't-propagate-tombstone:          (key=123, value=("don't propagate
>>>> me,
>>>>> but please delete state"))
>>>>>
>>>>> Would this pose some sort of verbosity problem in the internal topics,
>>>>> especially if we have to rebuild state off of them?
>>>>>
>>>>> Thanks
>>>>>
>>>>> Adam
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <
>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> SGTM.
>>>>>>
>>>>>> I also had the impression that those duplicates are rather an error
>>>> than
>>>>>> an case of eventual consistency. Using hashing to avoid sending the
>>>>>> payload is a good idea IMHO.
>>>>>>
>>>>>> @Adam: can you update the KIP accordingly?
>>>>>>
>>>>>>  - add the optimization to not send a reply from RHS to LHS on
>>>>>> unsubscribe (if not a tombstone)
>>>>>>  - explain why using offsets to avoid duplicates does not work
>>>>>>  - add hashing to avoid duplicates
>>>>>>
>>>>>> Beside this, I don't have any further comments. Excited to finally
>> get
>>>>>> this in!
>>>>>>
>>>>>> Let us know when you have updated the KIP so we can move forward with
>>>>>> the VOTE. Thanks a lot for your patience! This was a very loooong
>> shot!
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 3/8/19 8:47 AM, John Roesler wrote:
>>>>>>> Hi all,
>>>>>>>
>>>>>>> This proposal sounds good to me, especially since we observe that
>>>> people
>>>>>>> are already confused when the see duplicate results coming out of
>> 1:1
>>>>>> joins
>>>>>>> (which is a bug). I take this as "evidence" that we're better off
>>>>>>> eliminating those duplicates from the start. Guozhang's proposal
>> seems
>>>>>> like
>>>>>>> a lightweight solution to the problem, so FWIW, I'm in favor.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> -John
>>>>>>>
>>>>>>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
>>>> adam.bellemare@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Guozhang
>>>>>>>>
>>>>>>>> That would certainly work for eliminating those duplicate values.
>> As
>>>> it
>>>>>>>> stands right now, this would be consistent with swallowing changes
>>>> due
>>>>>> to
>>>>>>>> out-of-order processing with multiple threads, and seems like a
>> very
>>>>>>>> reasonable way forward. Thank you for the suggestion!
>>>>>>>>
>>>>>>>> I have been trying to think if there are any other scenarios where
>> we
>>>>>> can
>>>>>>>> end up with duplicates, though I have not been able to identify any
>>>>>> others
>>>>>>>> at the moment. I will think on it a bit more, but if anyone else
>> has
>>>> any
>>>>>>>> ideas, please chime in.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Adam
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
>>>>>> separate
>>>>>>>>> this issue with the more general issue that today (not only
>>>>>> foreign-key,
>>>>>>>>> but also co-partition primary-key) table-table joins is still not
>>>>>>>> strictly
>>>>>>>>> respecting the timestamp ordering since the two changelog streams
>>>> may
>>>>>> be
>>>>>>>>> fetched and hence processed out-of-order and we do not allow a
>>>> record
>>>>>> to
>>>>>>>> be
>>>>>>>>> joined with the other table at any given time snapshot yet. So
>>>> ideally
>>>>>>>> when
>>>>>>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
>>>>>> coming
>>>>>>>>> at the left hand table and one record (f-k1, v3) at the right hand
>>>>>> table,
>>>>>>>>> depending on the processing ordering we may get:
>>>>>>>>>
>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>
>>>>>>>>> or
>>>>>>>>>
>>>>>>>>> (k1, (f-k1, v1-v3))
>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>
>>>>>>>>> And this is not to be addressed by this KIP.
>>>>>>>>>
>>>>>>>>> What I would advocate is to fix the issue that is introduced in
>> this
>>>>>> KIP
>>>>>>>>> alone, that is we may have
>>>>>>>>>
>>>>>>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>
>>>>>>>>> I admit that it does not have correctness issue from the semantics
>>>>>> along,
>>>>>>>>> comparing it with "discarding the first result", but it may be
>>>>>> confusing
>>>>>>>>> from user's observation who do not expect to see the seemingly
>>>>>>>> duplicates.
>>>>>>>>> On the other hand, I think there's a light solution to avoid it,
>>>> which
>>>>>> is
>>>>>>>>> that we can still optimize away to not send the full payload of
>> "v1"
>>>>>> from
>>>>>>>>> left hand side to right hand side, but instead of just trimming
>> off
>>>> the
>>>>>>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm
>> using
>>>> MD5
>>>>>>>>> here just as an example, we can definitely replace it with other
>>>>>>>>> functions), by doing which we can discard the join operation if
>> the
>>>>>> hash
>>>>>>>>> value sent back from the right hand side does not match with the
>>>> left
>>>>>>>> hand
>>>>>>>>> side any more, i.e. we will only send:
>>>>>>>>>
>>>>>>>>> (k1, (f-k1, v2-v3))
>>>>>>>>>
>>>>>>>>> to down streams once.
>>>>>>>>>
>>>>>>>>> WDYT?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
>>>>>> adam.bellemare@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Ah yes, I recall it all now. That answers that question as to
>> why I
>>>>>> had
>>>>>>>>>> caching disabled. I can certainly re-enable it since I believe
>> the
>>>>>> main
>>>>>>>>>> concern was simply about reconciling those two iterators. A lack
>> of
>>>>>>>>>> knowledge there on my part.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thank you John for weighing in - we certainly both do appreciate
>>>> it. I
>>>>>>>>>> think that John hits it on the head though with his comment of
>> "If
>>>> it
>>>>>>>>> turns
>>>>>>>>>> out we're wrong about this, then it should be possible to fix the
>>>>>>>>> semantics
>>>>>>>>>> in place, without messing with the API."
>>>>>>>>>>
>>>>>>>>>> If anyone else would like to weigh in, your thoughts would be
>>>> greatly
>>>>>>>>>> appreciated.
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>>
>>>>>>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
>>>> matthias@confluent.io
>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>>>> I dont know how to range scan over a caching store, probably
>> one
>>>>>>>> had
>>>>>>>>>>>>> to open 2 iterators and merge them.
>>>>>>>>>>>
>>>>>>>>>>> That happens automatically. If you query a cached KTable, it
>>>> ranges
>>>>>>>>> over
>>>>>>>>>>> the cache and the underlying RocksDB and performs the merging
>>>> under
>>>>>>>> the
>>>>>>>>>>> hood.
>>>>>>>>>>>
>>>>>>>>>>>>> Other than that, I still think even the regualr join is broken
>>>>>>>> with
>>>>>>>>>>>>> caching enabled right?
>>>>>>>>>>>
>>>>>>>>>>> Why? To me, if you use the word "broker", it implies
>> conceptually
>>>>>>>>>>> incorrect; I don't see this.
>>>>>>>>>>>
>>>>>>>>>>>> I once files a ticket, because with caching
>>>>>>>>>>>>>> enabled it would return values that havent been published
>>>>>>>>> downstream
>>>>>>>>>>> yet.
>>>>>>>>>>>
>>>>>>>>>>> For the bug report, if found
>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need
>>>> to
>>>>>>>> fix
>>>>>>>>>>> this, but it is a regular bug as any other, and we should not
>>>> change
>>>>>>>> a
>>>>>>>>>>> design because of a bug.
>>>>>>>>>>>
>>>>>>>>>>> That range() returns values that have not been published
>>>> downstream
>>>>>>>> if
>>>>>>>>>>> caching is enabled is how caching works and is intended
>> behavior.
>>>> Not
>>>>>>>>>>> sure why say it's incorrect?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
>>>>>>>>>>>>> Thanks Adam,
>>>>>>>>>>>>>
>>>>>>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there
>> is
>>>>>>>> no
>>>>>>>>>>>>> functional/correctness requirement to disable caching. I
>> cannot
>>>>>>>>>>>>> remember why Jan's proposal added this? It might be an
>>>>>>>>>>>>> implementation detail though (maybe just remove it from the
>> KIP?
>>>>>>>>>>>>> -- might be miss leading).
>>>>>>>>>>>>
>>>>>>>>>>>> I dont know how to range scan over a caching store, probably
>> one
>>>>>>>> had
>>>>>>>>>>>> to open 2 iterators and merge them.
>>>>>>>>>>>>
>>>>>>>>>>>> Other than that, I still think even the regualr join is broken
>>>> with
>>>>>>>>>>>> caching enabled right? I once files a ticket, because with
>>>> caching
>>>>>>>>>>>> enabled it would return values that havent been published
>>>>>>>> downstream
>>>>>>>>>> yet.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> -- Guozhang
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hey Adam,

That's a good observation, but it wouldn't be a problem for repartition
topics because Streams aggressively deletes messages from the reparation
topics once it knows they are handled. Thus, we don't need to try and cater
to the log compactor.

Thanks,
-John

On Mon, Mar 11, 2019 at 9:10 AM Adam Bellemare <ad...@gmail.com>
wrote:

> For the sake of expediency, I updated the KIP with what I believe we have
> discussed.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges
>
>
>
> On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > My only concern was around compaction of records in the repartition
> topic.
> > This would simply mean that these records would stick around as their
> value
> > isn't truly null. Though I know about the usage of compaction on
> changelog
> > topics, I am a bit fuzzy on where we use compaction in other internal
> > topics. So long as this doesn't cause concern I can certainly finish off
> > the KIP today.
> >
> > Thanks
> >
> > Adam
> >
> > On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> I agree that the LHS side must encode this information and tell the RHS
> >> if a tombstone requires a reply or not.
> >>
> >> >> Would this pose some sort of verbosity problem in the internal
> topics,
> >> >> especially if we have to rebuild state off of them?
> >>
> >> I don't see an issue atm. Can you elaborate how this relates to rebuild
> >> state?
> >>
> >>
> >> -Matthias
> >>
> >> On 3/10/19 12:25 PM, Adam Bellemare wrote:
> >> > Hi Matthias
> >> >
> >> > I have been mulling over the unsubscribe / delete optimization, and I
> >> have
> >> > one main concern. I believe that the RHS can only determine whether to
> >> > propagate the tombstone or not based on the value passed over from the
> >> LHS.
> >> > This value would need to be non-null, and so wouldn't the internal
> >> > repartition topics end up containing many non-null "tombstone" values?
> >> >
> >> > ie:
> >> > Normal tombstone (propagate):     (key=123, value=null)
> >> > Don't-propagate-tombstone:          (key=123, value=("don't propagate
> >> me,
> >> > but please delete state"))
> >> >
> >> > Would this pose some sort of verbosity problem in the internal topics,
> >> > especially if we have to rebuild state off of them?
> >> >
> >> > Thanks
> >> >
> >> > Adam
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <
> matthias@confluent.io>
> >> > wrote:
> >> >
> >> >> SGTM.
> >> >>
> >> >> I also had the impression that those duplicates are rather an error
> >> than
> >> >> an case of eventual consistency. Using hashing to avoid sending the
> >> >> payload is a good idea IMHO.
> >> >>
> >> >> @Adam: can you update the KIP accordingly?
> >> >>
> >> >>  - add the optimization to not send a reply from RHS to LHS on
> >> >> unsubscribe (if not a tombstone)
> >> >>  - explain why using offsets to avoid duplicates does not work
> >> >>  - add hashing to avoid duplicates
> >> >>
> >> >> Beside this, I don't have any further comments. Excited to finally
> get
> >> >> this in!
> >> >>
> >> >> Let us know when you have updated the KIP so we can move forward with
> >> >> the VOTE. Thanks a lot for your patience! This was a very loooong
> shot!
> >> >>
> >> >>
> >> >> -Matthias
> >> >>
> >> >> On 3/8/19 8:47 AM, John Roesler wrote:
> >> >>> Hi all,
> >> >>>
> >> >>> This proposal sounds good to me, especially since we observe that
> >> people
> >> >>> are already confused when the see duplicate results coming out of
> 1:1
> >> >> joins
> >> >>> (which is a bug). I take this as "evidence" that we're better off
> >> >>> eliminating those duplicates from the start. Guozhang's proposal
> seems
> >> >> like
> >> >>> a lightweight solution to the problem, so FWIW, I'm in favor.
> >> >>>
> >> >>> Thanks,
> >> >>> -John
> >> >>>
> >> >>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >> >>> wrote:
> >> >>>
> >> >>>> Hi Guozhang
> >> >>>>
> >> >>>> That would certainly work for eliminating those duplicate values.
> As
> >> it
> >> >>>> stands right now, this would be consistent with swallowing changes
> >> due
> >> >> to
> >> >>>> out-of-order processing with multiple threads, and seems like a
> very
> >> >>>> reasonable way forward. Thank you for the suggestion!
> >> >>>>
> >> >>>> I have been trying to think if there are any other scenarios where
> we
> >> >> can
> >> >>>> end up with duplicates, though I have not been able to identify any
> >> >> others
> >> >>>> at the moment. I will think on it a bit more, but if anyone else
> has
> >> any
> >> >>>> ideas, please chime in.
> >> >>>>
> >> >>>> Thanks,
> >> >>>> Adam
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
> >> >> wrote:
> >> >>>>
> >> >>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
> >> >> separate
> >> >>>>> this issue with the more general issue that today (not only
> >> >> foreign-key,
> >> >>>>> but also co-partition primary-key) table-table joins is still not
> >> >>>> strictly
> >> >>>>> respecting the timestamp ordering since the two changelog streams
> >> may
> >> >> be
> >> >>>>> fetched and hence processed out-of-order and we do not allow a
> >> record
> >> >> to
> >> >>>> be
> >> >>>>> joined with the other table at any given time snapshot yet. So
> >> ideally
> >> >>>> when
> >> >>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
> >> >> coming
> >> >>>>> at the left hand table and one record (f-k1, v3) at the right hand
> >> >> table,
> >> >>>>> depending on the processing ordering we may get:
> >> >>>>>
> >> >>>>> (k1, (f-k1, v2-v3))
> >> >>>>>
> >> >>>>> or
> >> >>>>>
> >> >>>>> (k1, (f-k1, v1-v3))
> >> >>>>> (k1, (f-k1, v2-v3))
> >> >>>>>
> >> >>>>> And this is not to be addressed by this KIP.
> >> >>>>>
> >> >>>>> What I would advocate is to fix the issue that is introduced in
> this
> >> >> KIP
> >> >>>>> alone, that is we may have
> >> >>>>>
> >> >>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> >> >>>>> (k1, (f-k1, v2-v3))
> >> >>>>>
> >> >>>>> I admit that it does not have correctness issue from the semantics
> >> >> along,
> >> >>>>> comparing it with "discarding the first result", but it may be
> >> >> confusing
> >> >>>>> from user's observation who do not expect to see the seemingly
> >> >>>> duplicates.
> >> >>>>> On the other hand, I think there's a light solution to avoid it,
> >> which
> >> >> is
> >> >>>>> that we can still optimize away to not send the full payload of
> "v1"
> >> >> from
> >> >>>>> left hand side to right hand side, but instead of just trimming
> off
> >> the
> >> >>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm
> using
> >> MD5
> >> >>>>> here just as an example, we can definitely replace it with other
> >> >>>>> functions), by doing which we can discard the join operation if
> the
> >> >> hash
> >> >>>>> value sent back from the right hand side does not match with the
> >> left
> >> >>>> hand
> >> >>>>> side any more, i.e. we will only send:
> >> >>>>>
> >> >>>>> (k1, (f-k1, v2-v3))
> >> >>>>>
> >> >>>>> to down streams once.
> >> >>>>>
> >> >>>>> WDYT?
> >> >>>>>
> >> >>>>>
> >> >>>>> Guozhang
> >> >>>>>
> >> >>>>>
> >> >>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
> >> >> adam.bellemare@gmail.com>
> >> >>>>> wrote:
> >> >>>>>
> >> >>>>>> Ah yes, I recall it all now. That answers that question as to
> why I
> >> >> had
> >> >>>>>> caching disabled. I can certainly re-enable it since I believe
> the
> >> >> main
> >> >>>>>> concern was simply about reconciling those two iterators. A lack
> of
> >> >>>>>> knowledge there on my part.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> Thank you John for weighing in - we certainly both do appreciate
> >> it. I
> >> >>>>>> think that John hits it on the head though with his comment of
> "If
> >> it
> >> >>>>> turns
> >> >>>>>> out we're wrong about this, then it should be possible to fix the
> >> >>>>> semantics
> >> >>>>>> in place, without messing with the API."
> >> >>>>>>
> >> >>>>>> If anyone else would like to weigh in, your thoughts would be
> >> greatly
> >> >>>>>> appreciated.
> >> >>>>>>
> >> >>>>>> Thanks
> >> >>>>>>
> >> >>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
> >> matthias@confluent.io
> >> >>>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>>>> I dont know how to range scan over a caching store, probably
> one
> >> >>>> had
> >> >>>>>>>>> to open 2 iterators and merge them.
> >> >>>>>>>
> >> >>>>>>> That happens automatically. If you query a cached KTable, it
> >> ranges
> >> >>>>> over
> >> >>>>>>> the cache and the underlying RocksDB and performs the merging
> >> under
> >> >>>> the
> >> >>>>>>> hood.
> >> >>>>>>>
> >> >>>>>>>>> Other than that, I still think even the regualr join is broken
> >> >>>> with
> >> >>>>>>>>> caching enabled right?
> >> >>>>>>>
> >> >>>>>>> Why? To me, if you use the word "broker", it implies
> conceptually
> >> >>>>>>> incorrect; I don't see this.
> >> >>>>>>>
> >> >>>>>>>> I once files a ticket, because with caching
> >> >>>>>>>>>> enabled it would return values that havent been published
> >> >>>>> downstream
> >> >>>>>>> yet.
> >> >>>>>>>
> >> >>>>>>> For the bug report, if found
> >> >>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need
> >> to
> >> >>>> fix
> >> >>>>>>> this, but it is a regular bug as any other, and we should not
> >> change
> >> >>>> a
> >> >>>>>>> design because of a bug.
> >> >>>>>>>
> >> >>>>>>> That range() returns values that have not been published
> >> downstream
> >> >>>> if
> >> >>>>>>> caching is enabled is how caching works and is intended
> behavior.
> >> Not
> >> >>>>>>> sure why say it's incorrect?
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> -Matthias
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
> >> >>>>>>>>> Thanks Adam,
> >> >>>>>>>>>
> >> >>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there
> is
> >> >>>> no
> >> >>>>>>>>> functional/correctness requirement to disable caching. I
> cannot
> >> >>>>>>>>> remember why Jan's proposal added this? It might be an
> >> >>>>>>>>> implementation detail though (maybe just remove it from the
> KIP?
> >> >>>>>>>>> -- might be miss leading).
> >> >>>>>>>>
> >> >>>>>>>> I dont know how to range scan over a caching store, probably
> one
> >> >>>> had
> >> >>>>>>>> to open 2 iterators and merge them.
> >> >>>>>>>>
> >> >>>>>>>> Other than that, I still think even the regualr join is broken
> >> with
> >> >>>>>>>> caching enabled right? I once files a ticket, because with
> >> caching
> >> >>>>>>>> enabled it would return values that havent been published
> >> >>>> downstream
> >> >>>>>> yet.
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> --
> >> >>>>> -- Guozhang
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>
> >> >>
> >> >
> >>
> >>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
For the sake of expediency, I updated the KIP with what I believe we have
discussed.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Tombstones&ForeignKeyChanges



On Mon, Mar 11, 2019 at 8:20 AM Adam Bellemare <ad...@gmail.com>
wrote:

> My only concern was around compaction of records in the repartition topic.
> This would simply mean that these records would stick around as their value
> isn't truly null. Though I know about the usage of compaction on changelog
> topics, I am a bit fuzzy on where we use compaction in other internal
> topics. So long as this doesn't cause concern I can certainly finish off
> the KIP today.
>
> Thanks
>
> Adam
>
> On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> I agree that the LHS side must encode this information and tell the RHS
>> if a tombstone requires a reply or not.
>>
>> >> Would this pose some sort of verbosity problem in the internal topics,
>> >> especially if we have to rebuild state off of them?
>>
>> I don't see an issue atm. Can you elaborate how this relates to rebuild
>> state?
>>
>>
>> -Matthias
>>
>> On 3/10/19 12:25 PM, Adam Bellemare wrote:
>> > Hi Matthias
>> >
>> > I have been mulling over the unsubscribe / delete optimization, and I
>> have
>> > one main concern. I believe that the RHS can only determine whether to
>> > propagate the tombstone or not based on the value passed over from the
>> LHS.
>> > This value would need to be non-null, and so wouldn't the internal
>> > repartition topics end up containing many non-null "tombstone" values?
>> >
>> > ie:
>> > Normal tombstone (propagate):     (key=123, value=null)
>> > Don't-propagate-tombstone:          (key=123, value=("don't propagate
>> me,
>> > but please delete state"))
>> >
>> > Would this pose some sort of verbosity problem in the internal topics,
>> > especially if we have to rebuild state off of them?
>> >
>> > Thanks
>> >
>> > Adam
>> >
>> >
>> >
>> >
>> >
>> > On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> >
>> >> SGTM.
>> >>
>> >> I also had the impression that those duplicates are rather an error
>> than
>> >> an case of eventual consistency. Using hashing to avoid sending the
>> >> payload is a good idea IMHO.
>> >>
>> >> @Adam: can you update the KIP accordingly?
>> >>
>> >>  - add the optimization to not send a reply from RHS to LHS on
>> >> unsubscribe (if not a tombstone)
>> >>  - explain why using offsets to avoid duplicates does not work
>> >>  - add hashing to avoid duplicates
>> >>
>> >> Beside this, I don't have any further comments. Excited to finally get
>> >> this in!
>> >>
>> >> Let us know when you have updated the KIP so we can move forward with
>> >> the VOTE. Thanks a lot for your patience! This was a very loooong shot!
>> >>
>> >>
>> >> -Matthias
>> >>
>> >> On 3/8/19 8:47 AM, John Roesler wrote:
>> >>> Hi all,
>> >>>
>> >>> This proposal sounds good to me, especially since we observe that
>> people
>> >>> are already confused when the see duplicate results coming out of 1:1
>> >> joins
>> >>> (which is a bug). I take this as "evidence" that we're better off
>> >>> eliminating those duplicates from the start. Guozhang's proposal seems
>> >> like
>> >>> a lightweight solution to the problem, so FWIW, I'm in favor.
>> >>>
>> >>> Thanks,
>> >>> -John
>> >>>
>> >>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>> >>> wrote:
>> >>>
>> >>>> Hi Guozhang
>> >>>>
>> >>>> That would certainly work for eliminating those duplicate values. As
>> it
>> >>>> stands right now, this would be consistent with swallowing changes
>> due
>> >> to
>> >>>> out-of-order processing with multiple threads, and seems like a very
>> >>>> reasonable way forward. Thank you for the suggestion!
>> >>>>
>> >>>> I have been trying to think if there are any other scenarios where we
>> >> can
>> >>>> end up with duplicates, though I have not been able to identify any
>> >> others
>> >>>> at the moment. I will think on it a bit more, but if anyone else has
>> any
>> >>>> ideas, please chime in.
>> >>>>
>> >>>> Thanks,
>> >>>> Adam
>> >>>>
>> >>>>
>> >>>>
>> >>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
>> >> separate
>> >>>>> this issue with the more general issue that today (not only
>> >> foreign-key,
>> >>>>> but also co-partition primary-key) table-table joins is still not
>> >>>> strictly
>> >>>>> respecting the timestamp ordering since the two changelog streams
>> may
>> >> be
>> >>>>> fetched and hence processed out-of-order and we do not allow a
>> record
>> >> to
>> >>>> be
>> >>>>> joined with the other table at any given time snapshot yet. So
>> ideally
>> >>>> when
>> >>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
>> >> coming
>> >>>>> at the left hand table and one record (f-k1, v3) at the right hand
>> >> table,
>> >>>>> depending on the processing ordering we may get:
>> >>>>>
>> >>>>> (k1, (f-k1, v2-v3))
>> >>>>>
>> >>>>> or
>> >>>>>
>> >>>>> (k1, (f-k1, v1-v3))
>> >>>>> (k1, (f-k1, v2-v3))
>> >>>>>
>> >>>>> And this is not to be addressed by this KIP.
>> >>>>>
>> >>>>> What I would advocate is to fix the issue that is introduced in this
>> >> KIP
>> >>>>> alone, that is we may have
>> >>>>>
>> >>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
>> >>>>> (k1, (f-k1, v2-v3))
>> >>>>>
>> >>>>> I admit that it does not have correctness issue from the semantics
>> >> along,
>> >>>>> comparing it with "discarding the first result", but it may be
>> >> confusing
>> >>>>> from user's observation who do not expect to see the seemingly
>> >>>> duplicates.
>> >>>>> On the other hand, I think there's a light solution to avoid it,
>> which
>> >> is
>> >>>>> that we can still optimize away to not send the full payload of "v1"
>> >> from
>> >>>>> left hand side to right hand side, but instead of just trimming off
>> the
>> >>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using
>> MD5
>> >>>>> here just as an example, we can definitely replace it with other
>> >>>>> functions), by doing which we can discard the join operation if the
>> >> hash
>> >>>>> value sent back from the right hand side does not match with the
>> left
>> >>>> hand
>> >>>>> side any more, i.e. we will only send:
>> >>>>>
>> >>>>> (k1, (f-k1, v2-v3))
>> >>>>>
>> >>>>> to down streams once.
>> >>>>>
>> >>>>> WDYT?
>> >>>>>
>> >>>>>
>> >>>>> Guozhang
>> >>>>>
>> >>>>>
>> >>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
>> >> adam.bellemare@gmail.com>
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Ah yes, I recall it all now. That answers that question as to why I
>> >> had
>> >>>>>> caching disabled. I can certainly re-enable it since I believe the
>> >> main
>> >>>>>> concern was simply about reconciling those two iterators. A lack of
>> >>>>>> knowledge there on my part.
>> >>>>>>
>> >>>>>>
>> >>>>>> Thank you John for weighing in - we certainly both do appreciate
>> it. I
>> >>>>>> think that John hits it on the head though with his comment of "If
>> it
>> >>>>> turns
>> >>>>>> out we're wrong about this, then it should be possible to fix the
>> >>>>> semantics
>> >>>>>> in place, without messing with the API."
>> >>>>>>
>> >>>>>> If anyone else would like to weigh in, your thoughts would be
>> greatly
>> >>>>>> appreciated.
>> >>>>>>
>> >>>>>> Thanks
>> >>>>>>
>> >>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
>> matthias@confluent.io
>> >>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>>>> I dont know how to range scan over a caching store, probably one
>> >>>> had
>> >>>>>>>>> to open 2 iterators and merge them.
>> >>>>>>>
>> >>>>>>> That happens automatically. If you query a cached KTable, it
>> ranges
>> >>>>> over
>> >>>>>>> the cache and the underlying RocksDB and performs the merging
>> under
>> >>>> the
>> >>>>>>> hood.
>> >>>>>>>
>> >>>>>>>>> Other than that, I still think even the regualr join is broken
>> >>>> with
>> >>>>>>>>> caching enabled right?
>> >>>>>>>
>> >>>>>>> Why? To me, if you use the word "broker", it implies conceptually
>> >>>>>>> incorrect; I don't see this.
>> >>>>>>>
>> >>>>>>>> I once files a ticket, because with caching
>> >>>>>>>>>> enabled it would return values that havent been published
>> >>>>> downstream
>> >>>>>>> yet.
>> >>>>>>>
>> >>>>>>> For the bug report, if found
>> >>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need
>> to
>> >>>> fix
>> >>>>>>> this, but it is a regular bug as any other, and we should not
>> change
>> >>>> a
>> >>>>>>> design because of a bug.
>> >>>>>>>
>> >>>>>>> That range() returns values that have not been published
>> downstream
>> >>>> if
>> >>>>>>> caching is enabled is how caching works and is intended behavior.
>> Not
>> >>>>>>> sure why say it's incorrect?
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> -Matthias
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
>> >>>>>>>>> Thanks Adam,
>> >>>>>>>>>
>> >>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there is
>> >>>> no
>> >>>>>>>>> functional/correctness requirement to disable caching. I cannot
>> >>>>>>>>> remember why Jan's proposal added this? It might be an
>> >>>>>>>>> implementation detail though (maybe just remove it from the KIP?
>> >>>>>>>>> -- might be miss leading).
>> >>>>>>>>
>> >>>>>>>> I dont know how to range scan over a caching store, probably one
>> >>>> had
>> >>>>>>>> to open 2 iterators and merge them.
>> >>>>>>>>
>> >>>>>>>> Other than that, I still think even the regualr join is broken
>> with
>> >>>>>>>> caching enabled right? I once files a ticket, because with
>> caching
>> >>>>>>>> enabled it would return values that havent been published
>> >>>> downstream
>> >>>>>> yet.
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>>
>> >>>>> --
>> >>>>> -- Guozhang
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >>
>> >
>>
>>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
My only concern was around compaction of records in the repartition topic.
This would simply mean that these records would stick around as their value
isn't truly null. Though I know about the usage of compaction on changelog
topics, I am a bit fuzzy on where we use compaction in other internal
topics. So long as this doesn't cause concern I can certainly finish off
the KIP today.

Thanks

Adam

On Sun, Mar 10, 2019 at 11:38 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> I agree that the LHS side must encode this information and tell the RHS
> if a tombstone requires a reply or not.
>
> >> Would this pose some sort of verbosity problem in the internal topics,
> >> especially if we have to rebuild state off of them?
>
> I don't see an issue atm. Can you elaborate how this relates to rebuild
> state?
>
>
> -Matthias
>
> On 3/10/19 12:25 PM, Adam Bellemare wrote:
> > Hi Matthias
> >
> > I have been mulling over the unsubscribe / delete optimization, and I
> have
> > one main concern. I believe that the RHS can only determine whether to
> > propagate the tombstone or not based on the value passed over from the
> LHS.
> > This value would need to be non-null, and so wouldn't the internal
> > repartition topics end up containing many non-null "tombstone" values?
> >
> > ie:
> > Normal tombstone (propagate):     (key=123, value=null)
> > Don't-propagate-tombstone:          (key=123, value=("don't propagate me,
> > but please delete state"))
> >
> > Would this pose some sort of verbosity problem in the internal topics,
> > especially if we have to rebuild state off of them?
> >
> > Thanks
> >
> > Adam
> >
> >
> >
> >
> >
> > On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> SGTM.
> >>
> >> I also had the impression that those duplicates are rather an error than
> >> an case of eventual consistency. Using hashing to avoid sending the
> >> payload is a good idea IMHO.
> >>
> >> @Adam: can you update the KIP accordingly?
> >>
> >>  - add the optimization to not send a reply from RHS to LHS on
> >> unsubscribe (if not a tombstone)
> >>  - explain why using offsets to avoid duplicates does not work
> >>  - add hashing to avoid duplicates
> >>
> >> Beside this, I don't have any further comments. Excited to finally get
> >> this in!
> >>
> >> Let us know when you have updated the KIP so we can move forward with
> >> the VOTE. Thanks a lot for your patience! This was a very loooong shot!
> >>
> >>
> >> -Matthias
> >>
> >> On 3/8/19 8:47 AM, John Roesler wrote:
> >>> Hi all,
> >>>
> >>> This proposal sounds good to me, especially since we observe that
> people
> >>> are already confused when the see duplicate results coming out of 1:1
> >> joins
> >>> (which is a bug). I take this as "evidence" that we're better off
> >>> eliminating those duplicates from the start. Guozhang's proposal seems
> >> like
> >>> a lightweight solution to the problem, so FWIW, I'm in favor.
> >>>
> >>> Thanks,
> >>> -John
> >>>
> >>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <
> adam.bellemare@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Guozhang
> >>>>
> >>>> That would certainly work for eliminating those duplicate values. As
> it
> >>>> stands right now, this would be consistent with swallowing changes due
> >> to
> >>>> out-of-order processing with multiple threads, and seems like a very
> >>>> reasonable way forward. Thank you for the suggestion!
> >>>>
> >>>> I have been trying to think if there are any other scenarios where we
> >> can
> >>>> end up with duplicates, though I have not been able to identify any
> >> others
> >>>> at the moment. I will think on it a bit more, but if anyone else has
> any
> >>>> ideas, please chime in.
> >>>>
> >>>> Thanks,
> >>>> Adam
> >>>>
> >>>>
> >>>>
> >>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>>
> >>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
> >> separate
> >>>>> this issue with the more general issue that today (not only
> >> foreign-key,
> >>>>> but also co-partition primary-key) table-table joins is still not
> >>>> strictly
> >>>>> respecting the timestamp ordering since the two changelog streams may
> >> be
> >>>>> fetched and hence processed out-of-order and we do not allow a record
> >> to
> >>>> be
> >>>>> joined with the other table at any given time snapshot yet. So
> ideally
> >>>> when
> >>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
> >> coming
> >>>>> at the left hand table and one record (f-k1, v3) at the right hand
> >> table,
> >>>>> depending on the processing ordering we may get:
> >>>>>
> >>>>> (k1, (f-k1, v2-v3))
> >>>>>
> >>>>> or
> >>>>>
> >>>>> (k1, (f-k1, v1-v3))
> >>>>> (k1, (f-k1, v2-v3))
> >>>>>
> >>>>> And this is not to be addressed by this KIP.
> >>>>>
> >>>>> What I would advocate is to fix the issue that is introduced in this
> >> KIP
> >>>>> alone, that is we may have
> >>>>>
> >>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> >>>>> (k1, (f-k1, v2-v3))
> >>>>>
> >>>>> I admit that it does not have correctness issue from the semantics
> >> along,
> >>>>> comparing it with "discarding the first result", but it may be
> >> confusing
> >>>>> from user's observation who do not expect to see the seemingly
> >>>> duplicates.
> >>>>> On the other hand, I think there's a light solution to avoid it,
> which
> >> is
> >>>>> that we can still optimize away to not send the full payload of "v1"
> >> from
> >>>>> left hand side to right hand side, but instead of just trimming off
> the
> >>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using
> MD5
> >>>>> here just as an example, we can definitely replace it with other
> >>>>> functions), by doing which we can discard the join operation if the
> >> hash
> >>>>> value sent back from the right hand side does not match with the left
> >>>> hand
> >>>>> side any more, i.e. we will only send:
> >>>>>
> >>>>> (k1, (f-k1, v2-v3))
> >>>>>
> >>>>> to down streams once.
> >>>>>
> >>>>> WDYT?
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Ah yes, I recall it all now. That answers that question as to why I
> >> had
> >>>>>> caching disabled. I can certainly re-enable it since I believe the
> >> main
> >>>>>> concern was simply about reconciling those two iterators. A lack of
> >>>>>> knowledge there on my part.
> >>>>>>
> >>>>>>
> >>>>>> Thank you John for weighing in - we certainly both do appreciate
> it. I
> >>>>>> think that John hits it on the head though with his comment of "If
> it
> >>>>> turns
> >>>>>> out we're wrong about this, then it should be possible to fix the
> >>>>> semantics
> >>>>>> in place, without messing with the API."
> >>>>>>
> >>>>>> If anyone else would like to weigh in, your thoughts would be
> greatly
> >>>>>> appreciated.
> >>>>>>
> >>>>>> Thanks
> >>>>>>
> >>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <
> matthias@confluent.io
> >>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>>>> I dont know how to range scan over a caching store, probably one
> >>>> had
> >>>>>>>>> to open 2 iterators and merge them.
> >>>>>>>
> >>>>>>> That happens automatically. If you query a cached KTable, it ranges
> >>>>> over
> >>>>>>> the cache and the underlying RocksDB and performs the merging under
> >>>> the
> >>>>>>> hood.
> >>>>>>>
> >>>>>>>>> Other than that, I still think even the regualr join is broken
> >>>> with
> >>>>>>>>> caching enabled right?
> >>>>>>>
> >>>>>>> Why? To me, if you use the word "broker", it implies conceptually
> >>>>>>> incorrect; I don't see this.
> >>>>>>>
> >>>>>>>> I once files a ticket, because with caching
> >>>>>>>>>> enabled it would return values that havent been published
> >>>>> downstream
> >>>>>>> yet.
> >>>>>>>
> >>>>>>> For the bug report, if found
> >>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need to
> >>>> fix
> >>>>>>> this, but it is a regular bug as any other, and we should not
> change
> >>>> a
> >>>>>>> design because of a bug.
> >>>>>>>
> >>>>>>> That range() returns values that have not been published downstream
> >>>> if
> >>>>>>> caching is enabled is how caching works and is intended behavior.
> Not
> >>>>>>> sure why say it's incorrect?
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>>
> >>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
> >>>>>>>>> Thanks Adam,
> >>>>>>>>>
> >>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there is
> >>>> no
> >>>>>>>>> functional/correctness requirement to disable caching. I cannot
> >>>>>>>>> remember why Jan's proposal added this? It might be an
> >>>>>>>>> implementation detail though (maybe just remove it from the KIP?
> >>>>>>>>> -- might be miss leading).
> >>>>>>>>
> >>>>>>>> I dont know how to range scan over a caching store, probably one
> >>>> had
> >>>>>>>> to open 2 iterators and merge them.
> >>>>>>>>
> >>>>>>>> Other than that, I still think even the regualr join is broken
> with
> >>>>>>>> caching enabled right? I once files a ticket, because with caching
> >>>>>>>> enabled it would return values that havent been published
> >>>> downstream
> >>>>>> yet.
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>
> >>>
> >>
> >>
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I agree that the LHS side must encode this information and tell the RHS
if a tombstone requires a reply or not.

>> Would this pose some sort of verbosity problem in the internal topics,
>> especially if we have to rebuild state off of them?

I don't see an issue atm. Can you elaborate how this relates to rebuild
state?


-Matthias

On 3/10/19 12:25 PM, Adam Bellemare wrote:
> Hi Matthias
> 
> I have been mulling over the unsubscribe / delete optimization, and I have
> one main concern. I believe that the RHS can only determine whether to
> propagate the tombstone or not based on the value passed over from the LHS.
> This value would need to be non-null, and so wouldn't the internal
> repartition topics end up containing many non-null "tombstone" values?
> 
> ie:
> Normal tombstone (propagate):     (key=123, value=null)
> Don't-propagate-tombstone:          (key=123, value=("don't propagate me,
> but please delete state"))
> 
> Would this pose some sort of verbosity problem in the internal topics,
> especially if we have to rebuild state off of them?
> 
> Thanks
> 
> Adam
> 
> 
> 
> 
> 
> On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> SGTM.
>>
>> I also had the impression that those duplicates are rather an error than
>> an case of eventual consistency. Using hashing to avoid sending the
>> payload is a good idea IMHO.
>>
>> @Adam: can you update the KIP accordingly?
>>
>>  - add the optimization to not send a reply from RHS to LHS on
>> unsubscribe (if not a tombstone)
>>  - explain why using offsets to avoid duplicates does not work
>>  - add hashing to avoid duplicates
>>
>> Beside this, I don't have any further comments. Excited to finally get
>> this in!
>>
>> Let us know when you have updated the KIP so we can move forward with
>> the VOTE. Thanks a lot for your patience! This was a very loooong shot!
>>
>>
>> -Matthias
>>
>> On 3/8/19 8:47 AM, John Roesler wrote:
>>> Hi all,
>>>
>>> This proposal sounds good to me, especially since we observe that people
>>> are already confused when the see duplicate results coming out of 1:1
>> joins
>>> (which is a bug). I take this as "evidence" that we're better off
>>> eliminating those duplicates from the start. Guozhang's proposal seems
>> like
>>> a lightweight solution to the problem, so FWIW, I'm in favor.
>>>
>>> Thanks,
>>> -John
>>>
>>> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <ad...@gmail.com>
>>> wrote:
>>>
>>>> Hi Guozhang
>>>>
>>>> That would certainly work for eliminating those duplicate values. As it
>>>> stands right now, this would be consistent with swallowing changes due
>> to
>>>> out-of-order processing with multiple threads, and seems like a very
>>>> reasonable way forward. Thank you for the suggestion!
>>>>
>>>> I have been trying to think if there are any other scenarios where we
>> can
>>>> end up with duplicates, though I have not been able to identify any
>> others
>>>> at the moment. I will think on it a bit more, but if anyone else has any
>>>> ideas, please chime in.
>>>>
>>>> Thanks,
>>>> Adam
>>>>
>>>>
>>>>
>>>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>>
>>>>> One more thought regarding *c-P2: Duplicates)*: first I want to
>> separate
>>>>> this issue with the more general issue that today (not only
>> foreign-key,
>>>>> but also co-partition primary-key) table-table joins is still not
>>>> strictly
>>>>> respecting the timestamp ordering since the two changelog streams may
>> be
>>>>> fetched and hence processed out-of-order and we do not allow a record
>> to
>>>> be
>>>>> joined with the other table at any given time snapshot yet. So ideally
>>>> when
>>>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
>> coming
>>>>> at the left hand table and one record (f-k1, v3) at the right hand
>> table,
>>>>> depending on the processing ordering we may get:
>>>>>
>>>>> (k1, (f-k1, v2-v3))
>>>>>
>>>>> or
>>>>>
>>>>> (k1, (f-k1, v1-v3))
>>>>> (k1, (f-k1, v2-v3))
>>>>>
>>>>> And this is not to be addressed by this KIP.
>>>>>
>>>>> What I would advocate is to fix the issue that is introduced in this
>> KIP
>>>>> alone, that is we may have
>>>>>
>>>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
>>>>> (k1, (f-k1, v2-v3))
>>>>>
>>>>> I admit that it does not have correctness issue from the semantics
>> along,
>>>>> comparing it with "discarding the first result", but it may be
>> confusing
>>>>> from user's observation who do not expect to see the seemingly
>>>> duplicates.
>>>>> On the other hand, I think there's a light solution to avoid it, which
>> is
>>>>> that we can still optimize away to not send the full payload of "v1"
>> from
>>>>> left hand side to right hand side, but instead of just trimming off the
>>>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
>>>>> here just as an example, we can definitely replace it with other
>>>>> functions), by doing which we can discard the join operation if the
>> hash
>>>>> value sent back from the right hand side does not match with the left
>>>> hand
>>>>> side any more, i.e. we will only send:
>>>>>
>>>>> (k1, (f-k1, v2-v3))
>>>>>
>>>>> to down streams once.
>>>>>
>>>>> WDYT?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Ah yes, I recall it all now. That answers that question as to why I
>> had
>>>>>> caching disabled. I can certainly re-enable it since I believe the
>> main
>>>>>> concern was simply about reconciling those two iterators. A lack of
>>>>>> knowledge there on my part.
>>>>>>
>>>>>>
>>>>>> Thank you John for weighing in - we certainly both do appreciate it. I
>>>>>> think that John hits it on the head though with his comment of "If it
>>>>> turns
>>>>>> out we're wrong about this, then it should be possible to fix the
>>>>> semantics
>>>>>> in place, without messing with the API."
>>>>>>
>>>>>> If anyone else would like to weigh in, your thoughts would be greatly
>>>>>> appreciated.
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <matthias@confluent.io
>>>
>>>>>> wrote:
>>>>>>
>>>>>>>>> I dont know how to range scan over a caching store, probably one
>>>> had
>>>>>>>>> to open 2 iterators and merge them.
>>>>>>>
>>>>>>> That happens automatically. If you query a cached KTable, it ranges
>>>>> over
>>>>>>> the cache and the underlying RocksDB and performs the merging under
>>>> the
>>>>>>> hood.
>>>>>>>
>>>>>>>>> Other than that, I still think even the regualr join is broken
>>>> with
>>>>>>>>> caching enabled right?
>>>>>>>
>>>>>>> Why? To me, if you use the word "broker", it implies conceptually
>>>>>>> incorrect; I don't see this.
>>>>>>>
>>>>>>>> I once files a ticket, because with caching
>>>>>>>>>> enabled it would return values that havent been published
>>>>> downstream
>>>>>>> yet.
>>>>>>>
>>>>>>> For the bug report, if found
>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need to
>>>> fix
>>>>>>> this, but it is a regular bug as any other, and we should not change
>>>> a
>>>>>>> design because of a bug.
>>>>>>>
>>>>>>> That range() returns values that have not been published downstream
>>>> if
>>>>>>> caching is enabled is how caching works and is intended behavior. Not
>>>>>>> sure why say it's incorrect?
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
>>>>>>>>> Thanks Adam,
>>>>>>>>>
>>>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there is
>>>> no
>>>>>>>>> functional/correctness requirement to disable caching. I cannot
>>>>>>>>> remember why Jan's proposal added this? It might be an
>>>>>>>>> implementation detail though (maybe just remove it from the KIP?
>>>>>>>>> -- might be miss leading).
>>>>>>>>
>>>>>>>> I dont know how to range scan over a caching store, probably one
>>>> had
>>>>>>>> to open 2 iterators and merge them.
>>>>>>>>
>>>>>>>> Other than that, I still think even the regualr join is broken with
>>>>>>>> caching enabled right? I once files a ticket, because with caching
>>>>>>>> enabled it would return values that havent been published
>>>> downstream
>>>>>> yet.
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>
>>>
>>
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Matthias

I have been mulling over the unsubscribe / delete optimization, and I have
one main concern. I believe that the RHS can only determine whether to
propagate the tombstone or not based on the value passed over from the LHS.
This value would need to be non-null, and so wouldn't the internal
repartition topics end up containing many non-null "tombstone" values?

ie:
Normal tombstone (propagate):     (key=123, value=null)
Don't-propagate-tombstone:          (key=123, value=("don't propagate me,
but please delete state"))

Would this pose some sort of verbosity problem in the internal topics,
especially if we have to rebuild state off of them?

Thanks

Adam





On Fri, Mar 8, 2019 at 10:14 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> SGTM.
>
> I also had the impression that those duplicates are rather an error than
> an case of eventual consistency. Using hashing to avoid sending the
> payload is a good idea IMHO.
>
> @Adam: can you update the KIP accordingly?
>
>  - add the optimization to not send a reply from RHS to LHS on
> unsubscribe (if not a tombstone)
>  - explain why using offsets to avoid duplicates does not work
>  - add hashing to avoid duplicates
>
> Beside this, I don't have any further comments. Excited to finally get
> this in!
>
> Let us know when you have updated the KIP so we can move forward with
> the VOTE. Thanks a lot for your patience! This was a very loooong shot!
>
>
> -Matthias
>
> On 3/8/19 8:47 AM, John Roesler wrote:
> > Hi all,
> >
> > This proposal sounds good to me, especially since we observe that people
> > are already confused when the see duplicate results coming out of 1:1
> joins
> > (which is a bug). I take this as "evidence" that we're better off
> > eliminating those duplicates from the start. Guozhang's proposal seems
> like
> > a lightweight solution to the problem, so FWIW, I'm in favor.
> >
> > Thanks,
> > -John
> >
> > On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <ad...@gmail.com>
> > wrote:
> >
> >> Hi Guozhang
> >>
> >> That would certainly work for eliminating those duplicate values. As it
> >> stands right now, this would be consistent with swallowing changes due
> to
> >> out-of-order processing with multiple threads, and seems like a very
> >> reasonable way forward. Thank you for the suggestion!
> >>
> >> I have been trying to think if there are any other scenarios where we
> can
> >> end up with duplicates, though I have not been able to identify any
> others
> >> at the moment. I will think on it a bit more, but if anyone else has any
> >> ideas, please chime in.
> >>
> >> Thanks,
> >> Adam
> >>
> >>
> >>
> >> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> One more thought regarding *c-P2: Duplicates)*: first I want to
> separate
> >>> this issue with the more general issue that today (not only
> foreign-key,
> >>> but also co-partition primary-key) table-table joins is still not
> >> strictly
> >>> respecting the timestamp ordering since the two changelog streams may
> be
> >>> fetched and hence processed out-of-order and we do not allow a record
> to
> >> be
> >>> joined with the other table at any given time snapshot yet. So ideally
> >> when
> >>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2))
> coming
> >>> at the left hand table and one record (f-k1, v3) at the right hand
> table,
> >>> depending on the processing ordering we may get:
> >>>
> >>> (k1, (f-k1, v2-v3))
> >>>
> >>> or
> >>>
> >>> (k1, (f-k1, v1-v3))
> >>> (k1, (f-k1, v2-v3))
> >>>
> >>> And this is not to be addressed by this KIP.
> >>>
> >>> What I would advocate is to fix the issue that is introduced in this
> KIP
> >>> alone, that is we may have
> >>>
> >>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> >>> (k1, (f-k1, v2-v3))
> >>>
> >>> I admit that it does not have correctness issue from the semantics
> along,
> >>> comparing it with "discarding the first result", but it may be
> confusing
> >>> from user's observation who do not expect to see the seemingly
> >> duplicates.
> >>> On the other hand, I think there's a light solution to avoid it, which
> is
> >>> that we can still optimize away to not send the full payload of "v1"
> from
> >>> left hand side to right hand side, but instead of just trimming off the
> >>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
> >>> here just as an example, we can definitely replace it with other
> >>> functions), by doing which we can discard the join operation if the
> hash
> >>> value sent back from the right hand side does not match with the left
> >> hand
> >>> side any more, i.e. we will only send:
> >>>
> >>> (k1, (f-k1, v2-v3))
> >>>
> >>> to down streams once.
> >>>
> >>> WDYT?
> >>>
> >>>
> >>> Guozhang
> >>>
> >>>
> >>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <
> adam.bellemare@gmail.com>
> >>> wrote:
> >>>
> >>>> Ah yes, I recall it all now. That answers that question as to why I
> had
> >>>> caching disabled. I can certainly re-enable it since I believe the
> main
> >>>> concern was simply about reconciling those two iterators. A lack of
> >>>> knowledge there on my part.
> >>>>
> >>>>
> >>>> Thank you John for weighing in - we certainly both do appreciate it. I
> >>>> think that John hits it on the head though with his comment of "If it
> >>> turns
> >>>> out we're wrong about this, then it should be possible to fix the
> >>> semantics
> >>>> in place, without messing with the API."
> >>>>
> >>>> If anyone else would like to weigh in, your thoughts would be greatly
> >>>> appreciated.
> >>>>
> >>>> Thanks
> >>>>
> >>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <matthias@confluent.io
> >
> >>>> wrote:
> >>>>
> >>>>>>> I dont know how to range scan over a caching store, probably one
> >> had
> >>>>>>> to open 2 iterators and merge them.
> >>>>>
> >>>>> That happens automatically. If you query a cached KTable, it ranges
> >>> over
> >>>>> the cache and the underlying RocksDB and performs the merging under
> >> the
> >>>>> hood.
> >>>>>
> >>>>>>> Other than that, I still think even the regualr join is broken
> >> with
> >>>>>>> caching enabled right?
> >>>>>
> >>>>> Why? To me, if you use the word "broker", it implies conceptually
> >>>>> incorrect; I don't see this.
> >>>>>
> >>>>>> I once files a ticket, because with caching
> >>>>>>>> enabled it would return values that havent been published
> >>> downstream
> >>>>> yet.
> >>>>>
> >>>>> For the bug report, if found
> >>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need to
> >> fix
> >>>>> this, but it is a regular bug as any other, and we should not change
> >> a
> >>>>> design because of a bug.
> >>>>>
> >>>>> That range() returns values that have not been published downstream
> >> if
> >>>>> caching is enabled is how caching works and is intended behavior. Not
> >>>>> sure why say it's incorrect?
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> >>>>>>
> >>>>>>
> >>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
> >>>>>>> Thanks Adam,
> >>>>>>>
> >>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there is
> >> no
> >>>>>>> functional/correctness requirement to disable caching. I cannot
> >>>>>>> remember why Jan's proposal added this? It might be an
> >>>>>>> implementation detail though (maybe just remove it from the KIP?
> >>>>>>> -- might be miss leading).
> >>>>>>
> >>>>>> I dont know how to range scan over a caching store, probably one
> >> had
> >>>>>> to open 2 iterators and merge them.
> >>>>>>
> >>>>>> Other than that, I still think even the regualr join is broken with
> >>>>>> caching enabled right? I once files a ticket, because with caching
> >>>>>> enabled it would return values that havent been published
> >> downstream
> >>>> yet.
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
SGTM.

I also had the impression that those duplicates are rather an error than
an case of eventual consistency. Using hashing to avoid sending the
payload is a good idea IMHO.

@Adam: can you update the KIP accordingly?

 - add the optimization to not send a reply from RHS to LHS on
unsubscribe (if not a tombstone)
 - explain why using offsets to avoid duplicates does not work
 - add hashing to avoid duplicates

Beside this, I don't have any further comments. Excited to finally get
this in!

Let us know when you have updated the KIP so we can move forward with
the VOTE. Thanks a lot for your patience! This was a very loooong shot!


-Matthias

On 3/8/19 8:47 AM, John Roesler wrote:
> Hi all,
> 
> This proposal sounds good to me, especially since we observe that people
> are already confused when the see duplicate results coming out of 1:1 joins
> (which is a bug). I take this as "evidence" that we're better off
> eliminating those duplicates from the start. Guozhang's proposal seems like
> a lightweight solution to the problem, so FWIW, I'm in favor.
> 
> Thanks,
> -John
> 
> On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <ad...@gmail.com>
> wrote:
> 
>> Hi Guozhang
>>
>> That would certainly work for eliminating those duplicate values. As it
>> stands right now, this would be consistent with swallowing changes due to
>> out-of-order processing with multiple threads, and seems like a very
>> reasonable way forward. Thank you for the suggestion!
>>
>> I have been trying to think if there are any other scenarios where we can
>> end up with duplicates, though I have not been able to identify any others
>> at the moment. I will think on it a bit more, but if anyone else has any
>> ideas, please chime in.
>>
>> Thanks,
>> Adam
>>
>>
>>
>> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> One more thought regarding *c-P2: Duplicates)*: first I want to separate
>>> this issue with the more general issue that today (not only foreign-key,
>>> but also co-partition primary-key) table-table joins is still not
>> strictly
>>> respecting the timestamp ordering since the two changelog streams may be
>>> fetched and hence processed out-of-order and we do not allow a record to
>> be
>>> joined with the other table at any given time snapshot yet. So ideally
>> when
>>> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2)) coming
>>> at the left hand table and one record (f-k1, v3) at the right hand table,
>>> depending on the processing ordering we may get:
>>>
>>> (k1, (f-k1, v2-v3))
>>>
>>> or
>>>
>>> (k1, (f-k1, v1-v3))
>>> (k1, (f-k1, v2-v3))
>>>
>>> And this is not to be addressed by this KIP.
>>>
>>> What I would advocate is to fix the issue that is introduced in this KIP
>>> alone, that is we may have
>>>
>>> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
>>> (k1, (f-k1, v2-v3))
>>>
>>> I admit that it does not have correctness issue from the semantics along,
>>> comparing it with "discarding the first result", but it may be confusing
>>> from user's observation who do not expect to see the seemingly
>> duplicates.
>>> On the other hand, I think there's a light solution to avoid it, which is
>>> that we can still optimize away to not send the full payload of "v1" from
>>> left hand side to right hand side, but instead of just trimming off the
>>> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
>>> here just as an example, we can definitely replace it with other
>>> functions), by doing which we can discard the join operation if the hash
>>> value sent back from the right hand side does not match with the left
>> hand
>>> side any more, i.e. we will only send:
>>>
>>> (k1, (f-k1, v2-v3))
>>>
>>> to down streams once.
>>>
>>> WDYT?
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <ad...@gmail.com>
>>> wrote:
>>>
>>>> Ah yes, I recall it all now. That answers that question as to why I had
>>>> caching disabled. I can certainly re-enable it since I believe the main
>>>> concern was simply about reconciling those two iterators. A lack of
>>>> knowledge there on my part.
>>>>
>>>>
>>>> Thank you John for weighing in - we certainly both do appreciate it. I
>>>> think that John hits it on the head though with his comment of "If it
>>> turns
>>>> out we're wrong about this, then it should be possible to fix the
>>> semantics
>>>> in place, without messing with the API."
>>>>
>>>> If anyone else would like to weigh in, your thoughts would be greatly
>>>> appreciated.
>>>>
>>>> Thanks
>>>>
>>>> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>>>>> I dont know how to range scan over a caching store, probably one
>> had
>>>>>>> to open 2 iterators and merge them.
>>>>>
>>>>> That happens automatically. If you query a cached KTable, it ranges
>>> over
>>>>> the cache and the underlying RocksDB and performs the merging under
>> the
>>>>> hood.
>>>>>
>>>>>>> Other than that, I still think even the regualr join is broken
>> with
>>>>>>> caching enabled right?
>>>>>
>>>>> Why? To me, if you use the word "broker", it implies conceptually
>>>>> incorrect; I don't see this.
>>>>>
>>>>>> I once files a ticket, because with caching
>>>>>>>> enabled it would return values that havent been published
>>> downstream
>>>>> yet.
>>>>>
>>>>> For the bug report, if found
>>>>> https://issues.apache.org/jira/browse/KAFKA-6599. We still need to
>> fix
>>>>> this, but it is a regular bug as any other, and we should not change
>> a
>>>>> design because of a bug.
>>>>>
>>>>> That range() returns values that have not been published downstream
>> if
>>>>> caching is enabled is how caching works and is intended behavior. Not
>>>>> sure why say it's incorrect?
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 3/5/19 1:49 AM, Jan Filipiak wrote:
>>>>>>
>>>>>>
>>>>>> On 04.03.2019 19:14, Matthias J. Sax wrote:
>>>>>>> Thanks Adam,
>>>>>>>
>>>>>>> *> Q) Range scans work with caching enabled, too. Thus, there is
>> no
>>>>>>> functional/correctness requirement to disable caching. I cannot
>>>>>>> remember why Jan's proposal added this? It might be an
>>>>>>> implementation detail though (maybe just remove it from the KIP?
>>>>>>> -- might be miss leading).
>>>>>>
>>>>>> I dont know how to range scan over a caching store, probably one
>> had
>>>>>> to open 2 iterators and merge them.
>>>>>>
>>>>>> Other than that, I still think even the regualr join is broken with
>>>>>> caching enabled right? I once files a ticket, because with caching
>>>>>> enabled it would return values that havent been published
>> downstream
>>>> yet.
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi all,

This proposal sounds good to me, especially since we observe that people
are already confused when the see duplicate results coming out of 1:1 joins
(which is a bug). I take this as "evidence" that we're better off
eliminating those duplicates from the start. Guozhang's proposal seems like
a lightweight solution to the problem, so FWIW, I'm in favor.

Thanks,
-John

On Fri, Mar 8, 2019 at 7:59 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Guozhang
>
> That would certainly work for eliminating those duplicate values. As it
> stands right now, this would be consistent with swallowing changes due to
> out-of-order processing with multiple threads, and seems like a very
> reasonable way forward. Thank you for the suggestion!
>
> I have been trying to think if there are any other scenarios where we can
> end up with duplicates, though I have not been able to identify any others
> at the moment. I will think on it a bit more, but if anyone else has any
> ideas, please chime in.
>
> Thanks,
> Adam
>
>
>
> On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > One more thought regarding *c-P2: Duplicates)*: first I want to separate
> > this issue with the more general issue that today (not only foreign-key,
> > but also co-partition primary-key) table-table joins is still not
> strictly
> > respecting the timestamp ordering since the two changelog streams may be
> > fetched and hence processed out-of-order and we do not allow a record to
> be
> > joined with the other table at any given time snapshot yet. So ideally
> when
> > there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2)) coming
> > at the left hand table and one record (f-k1, v3) at the right hand table,
> > depending on the processing ordering we may get:
> >
> > (k1, (f-k1, v2-v3))
> >
> > or
> >
> > (k1, (f-k1, v1-v3))
> > (k1, (f-k1, v2-v3))
> >
> > And this is not to be addressed by this KIP.
> >
> > What I would advocate is to fix the issue that is introduced in this KIP
> > alone, that is we may have
> >
> > (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> > (k1, (f-k1, v2-v3))
> >
> > I admit that it does not have correctness issue from the semantics along,
> > comparing it with "discarding the first result", but it may be confusing
> > from user's observation who do not expect to see the seemingly
> duplicates.
> > On the other hand, I think there's a light solution to avoid it, which is
> > that we can still optimize away to not send the full payload of "v1" from
> > left hand side to right hand side, but instead of just trimming off the
> > whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
> > here just as an example, we can definitely replace it with other
> > functions), by doing which we can discard the join operation if the hash
> > value sent back from the right hand side does not match with the left
> hand
> > side any more, i.e. we will only send:
> >
> > (k1, (f-k1, v2-v3))
> >
> > to down streams once.
> >
> > WDYT?
> >
> >
> > Guozhang
> >
> >
> > On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <ad...@gmail.com>
> > wrote:
> >
> > > Ah yes, I recall it all now. That answers that question as to why I had
> > > caching disabled. I can certainly re-enable it since I believe the main
> > > concern was simply about reconciling those two iterators. A lack of
> > > knowledge there on my part.
> > >
> > >
> > > Thank you John for weighing in - we certainly both do appreciate it. I
> > > think that John hits it on the head though with his comment of "If it
> > turns
> > > out we're wrong about this, then it should be possible to fix the
> > semantics
> > > in place, without messing with the API."
> > >
> > > If anyone else would like to weigh in, your thoughts would be greatly
> > > appreciated.
> > >
> > > Thanks
> > >
> > > On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <ma...@confluent.io>
> > > wrote:
> > >
> > > > >> I dont know how to range scan over a caching store, probably one
> had
> > > > >> to open 2 iterators and merge them.
> > > >
> > > > That happens automatically. If you query a cached KTable, it ranges
> > over
> > > > the cache and the underlying RocksDB and performs the merging under
> the
> > > > hood.
> > > >
> > > > >> Other than that, I still think even the regualr join is broken
> with
> > > > >> caching enabled right?
> > > >
> > > > Why? To me, if you use the word "broker", it implies conceptually
> > > > incorrect; I don't see this.
> > > >
> > > > > I once files a ticket, because with caching
> > > > >>> enabled it would return values that havent been published
> > downstream
> > > > yet.
> > > >
> > > > For the bug report, if found
> > > > https://issues.apache.org/jira/browse/KAFKA-6599. We still need to
> fix
> > > > this, but it is a regular bug as any other, and we should not change
> a
> > > > design because of a bug.
> > > >
> > > > That range() returns values that have not been published downstream
> if
> > > > caching is enabled is how caching works and is intended behavior. Not
> > > > sure why say it's incorrect?
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > > On 3/5/19 1:49 AM, Jan Filipiak wrote:
> > > > >
> > > > >
> > > > > On 04.03.2019 19:14, Matthias J. Sax wrote:
> > > > >> Thanks Adam,
> > > > >>
> > > > >> *> Q) Range scans work with caching enabled, too. Thus, there is
> no
> > > > >> functional/correctness requirement to disable caching. I cannot
> > > > >> remember why Jan's proposal added this? It might be an
> > > > >> implementation detail though (maybe just remove it from the KIP?
> > > > >> -- might be miss leading).
> > > > >
> > > > > I dont know how to range scan over a caching store, probably one
> had
> > > > > to open 2 iterators and merge them.
> > > > >
> > > > > Other than that, I still think even the regualr join is broken with
> > > > > caching enabled right? I once files a ticket, because with caching
> > > > > enabled it would return values that havent been published
> downstream
> > > yet.
> > > > >
> > > >
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Guozhang

That would certainly work for eliminating those duplicate values. As it
stands right now, this would be consistent with swallowing changes due to
out-of-order processing with multiple threads, and seems like a very
reasonable way forward. Thank you for the suggestion!

I have been trying to think if there are any other scenarios where we can
end up with duplicates, though I have not been able to identify any others
at the moment. I will think on it a bit more, but if anyone else has any
ideas, please chime in.

Thanks,
Adam



On Thu, Mar 7, 2019 at 8:19 PM Guozhang Wang <wa...@gmail.com> wrote:

> One more thought regarding *c-P2: Duplicates)*: first I want to separate
> this issue with the more general issue that today (not only foreign-key,
> but also co-partition primary-key) table-table joins is still not strictly
> respecting the timestamp ordering since the two changelog streams may be
> fetched and hence processed out-of-order and we do not allow a record to be
> joined with the other table at any given time snapshot yet. So ideally when
> there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2)) coming
> at the left hand table and one record (f-k1, v3) at the right hand table,
> depending on the processing ordering we may get:
>
> (k1, (f-k1, v2-v3))
>
> or
>
> (k1, (f-k1, v1-v3))
> (k1, (f-k1, v2-v3))
>
> And this is not to be addressed by this KIP.
>
> What I would advocate is to fix the issue that is introduced in this KIP
> alone, that is we may have
>
> (k1, (f-k1, v2-v3))   // this should actually be v1-v3
> (k1, (f-k1, v2-v3))
>
> I admit that it does not have correctness issue from the semantics along,
> comparing it with "discarding the first result", but it may be confusing
> from user's observation who do not expect to see the seemingly duplicates.
> On the other hand, I think there's a light solution to avoid it, which is
> that we can still optimize away to not send the full payload of "v1" from
> left hand side to right hand side, but instead of just trimming off the
> whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
> here just as an example, we can definitely replace it with other
> functions), by doing which we can discard the join operation if the hash
> value sent back from the right hand side does not match with the left hand
> side any more, i.e. we will only send:
>
> (k1, (f-k1, v2-v3))
>
> to down streams once.
>
> WDYT?
>
>
> Guozhang
>
>
> On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > Ah yes, I recall it all now. That answers that question as to why I had
> > caching disabled. I can certainly re-enable it since I believe the main
> > concern was simply about reconciling those two iterators. A lack of
> > knowledge there on my part.
> >
> >
> > Thank you John for weighing in - we certainly both do appreciate it. I
> > think that John hits it on the head though with his comment of "If it
> turns
> > out we're wrong about this, then it should be possible to fix the
> semantics
> > in place, without messing with the API."
> >
> > If anyone else would like to weigh in, your thoughts would be greatly
> > appreciated.
> >
> > Thanks
> >
> > On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > >> I dont know how to range scan over a caching store, probably one had
> > > >> to open 2 iterators and merge them.
> > >
> > > That happens automatically. If you query a cached KTable, it ranges
> over
> > > the cache and the underlying RocksDB and performs the merging under the
> > > hood.
> > >
> > > >> Other than that, I still think even the regualr join is broken with
> > > >> caching enabled right?
> > >
> > > Why? To me, if you use the word "broker", it implies conceptually
> > > incorrect; I don't see this.
> > >
> > > > I once files a ticket, because with caching
> > > >>> enabled it would return values that havent been published
> downstream
> > > yet.
> > >
> > > For the bug report, if found
> > > https://issues.apache.org/jira/browse/KAFKA-6599. We still need to fix
> > > this, but it is a regular bug as any other, and we should not change a
> > > design because of a bug.
> > >
> > > That range() returns values that have not been published downstream if
> > > caching is enabled is how caching works and is intended behavior. Not
> > > sure why say it's incorrect?
> > >
> > >
> > > -Matthias
> > >
> > >
> > > On 3/5/19 1:49 AM, Jan Filipiak wrote:
> > > >
> > > >
> > > > On 04.03.2019 19:14, Matthias J. Sax wrote:
> > > >> Thanks Adam,
> > > >>
> > > >> *> Q) Range scans work with caching enabled, too. Thus, there is no
> > > >> functional/correctness requirement to disable caching. I cannot
> > > >> remember why Jan's proposal added this? It might be an
> > > >> implementation detail though (maybe just remove it from the KIP?
> > > >> -- might be miss leading).
> > > >
> > > > I dont know how to range scan over a caching store, probably one had
> > > > to open 2 iterators and merge them.
> > > >
> > > > Other than that, I still think even the regualr join is broken with
> > > > caching enabled right? I once files a ticket, because with caching
> > > > enabled it would return values that havent been published downstream
> > yet.
> > > >
> > >
> > >
> >
>
>
> --
> -- Guozhang
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Guozhang Wang <wa...@gmail.com>.
One more thought regarding *c-P2: Duplicates)*: first I want to separate
this issue with the more general issue that today (not only foreign-key,
but also co-partition primary-key) table-table joins is still not strictly
respecting the timestamp ordering since the two changelog streams may be
fetched and hence processed out-of-order and we do not allow a record to be
joined with the other table at any given time snapshot yet. So ideally when
there are two changelog records (k1, (f-k1, v1)), (k1, (f-k1, v2)) coming
at the left hand table and one record (f-k1, v3) at the right hand table,
depending on the processing ordering we may get:

(k1, (f-k1, v2-v3))

or

(k1, (f-k1, v1-v3))
(k1, (f-k1, v2-v3))

And this is not to be addressed by this KIP.

What I would advocate is to fix the issue that is introduced in this KIP
alone, that is we may have

(k1, (f-k1, v2-v3))   // this should actually be v1-v3
(k1, (f-k1, v2-v3))

I admit that it does not have correctness issue from the semantics along,
comparing it with "discarding the first result", but it may be confusing
from user's observation who do not expect to see the seemingly duplicates.
On the other hand, I think there's a light solution to avoid it, which is
that we can still optimize away to not send the full payload of "v1" from
left hand side to right hand side, but instead of just trimming off the
whole bytes, we can send, e.g., an MD5 hash of the bytes (I'm using MD5
here just as an example, we can definitely replace it with other
functions), by doing which we can discard the join operation if the hash
value sent back from the right hand side does not match with the left hand
side any more, i.e. we will only send:

(k1, (f-k1, v2-v3))

to down streams once.

WDYT?


Guozhang


On Wed, Mar 6, 2019 at 7:58 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Ah yes, I recall it all now. That answers that question as to why I had
> caching disabled. I can certainly re-enable it since I believe the main
> concern was simply about reconciling those two iterators. A lack of
> knowledge there on my part.
>
>
> Thank you John for weighing in - we certainly both do appreciate it. I
> think that John hits it on the head though with his comment of "If it turns
> out we're wrong about this, then it should be possible to fix the semantics
> in place, without messing with the API."
>
> If anyone else would like to weigh in, your thoughts would be greatly
> appreciated.
>
> Thanks
>
> On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > >> I dont know how to range scan over a caching store, probably one had
> > >> to open 2 iterators and merge them.
> >
> > That happens automatically. If you query a cached KTable, it ranges over
> > the cache and the underlying RocksDB and performs the merging under the
> > hood.
> >
> > >> Other than that, I still think even the regualr join is broken with
> > >> caching enabled right?
> >
> > Why? To me, if you use the word "broker", it implies conceptually
> > incorrect; I don't see this.
> >
> > > I once files a ticket, because with caching
> > >>> enabled it would return values that havent been published downstream
> > yet.
> >
> > For the bug report, if found
> > https://issues.apache.org/jira/browse/KAFKA-6599. We still need to fix
> > this, but it is a regular bug as any other, and we should not change a
> > design because of a bug.
> >
> > That range() returns values that have not been published downstream if
> > caching is enabled is how caching works and is intended behavior. Not
> > sure why say it's incorrect?
> >
> >
> > -Matthias
> >
> >
> > On 3/5/19 1:49 AM, Jan Filipiak wrote:
> > >
> > >
> > > On 04.03.2019 19:14, Matthias J. Sax wrote:
> > >> Thanks Adam,
> > >>
> > >> *> Q) Range scans work with caching enabled, too. Thus, there is no
> > >> functional/correctness requirement to disable caching. I cannot
> > >> remember why Jan's proposal added this? It might be an
> > >> implementation detail though (maybe just remove it from the KIP?
> > >> -- might be miss leading).
> > >
> > > I dont know how to range scan over a caching store, probably one had
> > > to open 2 iterators and merge them.
> > >
> > > Other than that, I still think even the regualr join is broken with
> > > caching enabled right? I once files a ticket, because with caching
> > > enabled it would return values that havent been published downstream
> yet.
> > >
> >
> >
>


-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Ah yes, I recall it all now. That answers that question as to why I had
caching disabled. I can certainly re-enable it since I believe the main
concern was simply about reconciling those two iterators. A lack of
knowledge there on my part.


Thank you John for weighing in - we certainly both do appreciate it. I
think that John hits it on the head though with his comment of "If it turns
out we're wrong about this, then it should be possible to fix the semantics
in place, without messing with the API."

If anyone else would like to weigh in, your thoughts would be greatly
appreciated.

Thanks

On Tue, Mar 5, 2019 at 6:05 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> >> I dont know how to range scan over a caching store, probably one had
> >> to open 2 iterators and merge them.
>
> That happens automatically. If you query a cached KTable, it ranges over
> the cache and the underlying RocksDB and performs the merging under the
> hood.
>
> >> Other than that, I still think even the regualr join is broken with
> >> caching enabled right?
>
> Why? To me, if you use the word "broker", it implies conceptually
> incorrect; I don't see this.
>
> > I once files a ticket, because with caching
> >>> enabled it would return values that havent been published downstream
> yet.
>
> For the bug report, if found
> https://issues.apache.org/jira/browse/KAFKA-6599. We still need to fix
> this, but it is a regular bug as any other, and we should not change a
> design because of a bug.
>
> That range() returns values that have not been published downstream if
> caching is enabled is how caching works and is intended behavior. Not
> sure why say it's incorrect?
>
>
> -Matthias
>
>
> On 3/5/19 1:49 AM, Jan Filipiak wrote:
> >
> >
> > On 04.03.2019 19:14, Matthias J. Sax wrote:
> >> Thanks Adam,
> >>
> >> *> Q) Range scans work with caching enabled, too. Thus, there is no
> >> functional/correctness requirement to disable caching. I cannot
> >> remember why Jan's proposal added this? It might be an
> >> implementation detail though (maybe just remove it from the KIP?
> >> -- might be miss leading).
> >
> > I dont know how to range scan over a caching store, probably one had
> > to open 2 iterators and merge them.
> >
> > Other than that, I still think even the regualr join is broken with
> > caching enabled right? I once files a ticket, because with caching
> > enabled it would return values that havent been published downstream yet.
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
>> I dont know how to range scan over a caching store, probably one had
>> to open 2 iterators and merge them.

That happens automatically. If you query a cached KTable, it ranges over
the cache and the underlying RocksDB and performs the merging under the
hood.

>> Other than that, I still think even the regualr join is broken with
>> caching enabled right?

Why? To me, if you use the word "broker", it implies conceptually
incorrect; I don't see this.

> I once files a ticket, because with caching
>>> enabled it would return values that havent been published downstream yet.

For the bug report, if found
https://issues.apache.org/jira/browse/KAFKA-6599. We still need to fix
this, but it is a regular bug as any other, and we should not change a
design because of a bug.

That range() returns values that have not been published downstream if
caching is enabled is how caching works and is intended behavior. Not
sure why say it's incorrect?


-Matthias


On 3/5/19 1:49 AM, Jan Filipiak wrote:
> 
> 
> On 04.03.2019 19:14, Matthias J. Sax wrote:
>> Thanks Adam,
>>
>> *> Q) Range scans work with caching enabled, too. Thus, there is no
>> functional/correctness requirement to disable caching. I cannot 
>> remember why Jan's proposal added this? It might be an 
>> implementation detail though (maybe just remove it from the KIP?
>> -- might be miss leading).
> 
> I dont know how to range scan over a caching store, probably one had
> to open 2 iterators and merge them.
> 
> Other than that, I still think even the regualr join is broken with
> caching enabled right? I once files a ticket, because with caching
> enabled it would return values that havent been published downstream yet.
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

On 04.03.2019 19:14, Matthias J. Sax wrote:
> Thanks Adam,
> 
> *> Q) Range scans work with caching enabled, too. Thus, there is no
> functional/correctness requirement to disable caching. I cannot 
> remember why Jan's proposal added this? It might be an 
> implementation detail though (maybe just remove it from the KIP?
> -- might be miss leading).

I dont know how to range scan over a caching store, probably one had
to open 2 iterators and merge them.

Other than that, I still think even the regualr join is broken with
caching enabled right? I once files a ticket, because with caching
enabled it would return values that havent been published downstream yet.

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi all,

I don't have too much time to respond at the moment, but I don't have much
to say anyway. I just wanted to respond to your requests for more opinions.

The point Matthias brought up is a good one. I think this was similar to
Jan's concern before about "swallowing" some intermediate states. Adam's
example of triggering logic based on the state transitions of the KTables
is a plausibe (if advanced and exotic) case where it would matter. For all
"pure" relational-algebra-type use cases, I *think* the proposed,
eventually-correct, solution would be fine.

I think both the "swallowing intermediate results" and the "duplicate final
results" problems are probably possible to solve, but my instinct says that
it would be costly, and what we have proposed right now probably satisfies
the majority use case. If it turns out we're wrong about this, then it
should be possible to fix the semantics in place, without messing with the
API.

Thanks, everyone,
-John

On Mon, Mar 4, 2019 at 12:23 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Thanks Adam,
>
> *> Q)
> Range scans work with caching enabled, too. Thus, there is no
> functional/correctness requirement to disable caching. I cannot remember
> why Jan's proposal added this? It might be an implementation detail
> though (maybe just remove it from the KIP? -- might be miss leading).
>
>
> *> (b)
> For stream.flatMap()... There is only one input record for the LHS. This
> single input record, may produce multiple updates to the KTable --
> however, those update records are not written into a topic (before the
> update) and thus they don't have their own offsets (if you call
> context.offset() for them, you will get the offset of the original input
> record, ie, the same offset for all the time). Of course, on write into
> the first repartition topic they will get their own offsets. However,
> this information seems not to be useful, because this offsets are not
> related to the input topic offset and might be from different
> partitions. Thus, I don't see how reordering could be resolves using
> them? However, I don't think we need offsets anyway to resolve ordering
> issues, thus, it does not really matter. (I added this just as a side
> remark in my response.)
>
>
> *c-P1)*
> *c-P2: Duplicates)*
> I see your point here and agree that it might be difficult to send _all_
> result updates if the RHS processors have different load and one might
> lag. However, "skipping" some intermediate updates seems to be something
> different that "duplicating" an update? However, I also agree that
> sending the payload to the RHS might be cost prohibitive. For your
> filter example, it seems that calling `KTable.toStream().filter()` would
> be incorrect, but that it should be `KTable.filter()`. For the first
> scenario I agree that the output record would never make it into the
> result, however, for the second case, update semantics are preserved and
> thus the result would still be correct (the output record won't be in
> the result KTable, either, but this would be correct! -- note that
> calling `KTable.toStream()` changes the semantics(!)).
>
> Additionally, as you mentioned ("Monkey Wrench"), there are still other
> scenario for which intermediate update might get swallowed. This seems
> to be a general "issue" in the current design. Maybe that was the main
> criticism from Jan (and it is a quite valid point). However, which
> caching enabled, swallowing updates is intended behavior anyway, so I
> personally don't think this is a big concern atm. (This might go back to
> Jan's proposal to disable caching to avoid swallowing any updates?)
>
> -> would be good to get input from others about the "swallowing" issue
>
> Thus, I agree with your conclusion that sending the payload to RHS does
> not resolve this issue. However, I still believe that the current design
> ensures that we don't overwrite the correct final result with a stale
> intermediate result. The FK check before the result data is emitted
> seems to be sufficient for that (I don't think we need timestamp or
> offsets to resolve this). The FK check has the only issue, that it may
> produce "duplicate" updates. Or did I miss anything?
>
> -> would be good to get input from others about this "duplicate update"
> issue
>
>
>
> -Matthias
>
>
>
>
> On 3/4/19 8:29 AM, Adam Bellemare wrote:
> > Hi Matthias
> >
> > Thank you for the feedback! I appreciate your well thought-out
> questions. I
> > have tried to answer and comment on everything that I know below.
> >
> >
> >
> > *> Q) For the materialized combined-key store, why do we need to disable>
> > caching? And why do we need to flush the store?*
> > This is an artifact from Jan's implementation that I have carried along.
> My
> > understanding (though possibly erroneous!) is that RocksDB prefix scan
> > doesn't work with the cache, and ignores any data stored within it. I
> have
> > tried to validate this but I have not succeeded, so I believe that this
> > will need more investigation and testing. I will dig deeper on this and
> get
> > back to you.
> >
> >
> >
> > *> a) Thus, I am wondering why we would need to send the `null` message
> > back> (from RHS to LHS) in the first place?*
> >
> > We don't need to, if we follow your subsequent tombstone suggestion.
> >
> >
> >
> >
> >
> > *> (b) About using "offsets" to resolve ordering issue: I don't think
> this>
> > would work. The join input table would be created as>
> > stream.flatMapValues().groupByKey().aggregate()*
> > Hmmm... I am a bit fuzzy on this part. Wouldn't the LHS processor be able
> > to get the highest offset and propagate that onwards to the RHS
> processor?
> > In my original design I had a wrapper that kept track of the input
> offset,
> > though I suspect it did not work for the above aggregation scenario.
> >
> > *c-P1)*
> > Regarding the consistency examples, everything you wrote is correct as
> far
> > as I can tell in how the proposed system would behave. Rapid updates to
> the
> > LHS will result in some of the results being discarded (in the case of
> > deletes or change of FK) or doubly-produced (discussed below, after the
> > following example).
> >
> > It does not seem to me to be possible to avoid quashing records that are
> > late arriving from the RHS. This could commonly be exhibited by two RHS
> > processors that are receiving very different loads. In the example below,
> > consider RHS-1 to be heavily loaded while RHS-2 is idle.
> >
> > Example:
> > 1- RHS-1 is updated to Y|bar
> > 2- RHS-2 is updated to Z|foo
> > 3- LHS is updated to A|Y
> >    -> sends Y|A+ subscription message to RHS-1
> > 3- LHS is updated to A|Z
> >    -> sends Y|A- unsubscribe message to RHS-1
> >    -> sends Z|A+ subscription to RHS-2
> > 4- RHS-2 processes Z|A message immediately
> >    -> sends A|Z,foo back
> > 5- LHS processes A|Z,foo and produces result record A|join(Z,foo)
> > 4- RHS-1 processes Y|A message
> >    -> sends A|Y,bar back
> > 4- RHS-1 processes Y|A- unsubscribe message
> >    -> sends A|null message back
> > X- LHS processes A|Y,bar, compares it to A|Z, and discards it due to
> > staleness.
> > X- LHS processes A|null, compares it to A|Z, and discards it due to
> > staleness.
> >
> > In this case, intermediate messages were discarded due to staleness. From
> > the outside, this can be seen as "incorrect" because these intermediate
> > results were not shown. However, there is no possible way for RHS-2 to
> know
> > to delay production of its event until RHS-1 has completed its
> > propagations. If we wish to produce all intermediate events, in order, we
> > must maintain state on the LHS about which events have been sent out,
> await
> > their return, and only publish them in order. Aside from the obvious
> > complexity and memory requirements, the intermediate events would
> > immediately be stomped by the final output.
> >
> >
> > *c-P2: Duplicates)*
> > With regards to duplicates (as per the double-sending of `A|Y,2,bar`),
> one
> > option is to ship the entire payload of the LHS over to the RHS, and
> either
> > join there or ship the entire payload back along with the RHS record. We
> > would still need to compare the FK on the LHS to ensure that it is still
> > valid. To take your example and expand it:
> >
> > 1- RHS is updated to Y|bar
> > 2- LHS is updated to A|Y,1
> >    -> sends Y|(A, (Y,1))+ subscription message to RHS
> > 3- LHS is updated to A|Y,2
> >    -> sends Y|(A, (Y,1))- unsubscribe message to RHS
> >    -> sends Y|(A, (Y,2))+ subscription to RHS
> > 4- RHS processes first Y|A+ message
> >    -> sends A|(A, (Y,1)),bar back
> > 5- LHS processes A|(A, (Y,1)),bar and produces result record A|Y,1,bar
> > 6- RHS processes Y|(A, (Y,1))- unsubscribe message (update store only)
> > 7- RHS processes second Y|(A, (Y,2))+ subscribe message
> >    -> sends A|(A, (Y,2)),bar back
> > 8- LHS processes A|(A, (Y,2)),bar and produces result record A|Y,2,bar
> >
> > Thus, the first result record is now `A|Y,1,bar`, while the second is
> > `A|Y,2,bar`.
> >
> > This will add substantially to the data payload size. The question here
> > then becomes, "In which scenario is this a necessity"?
> >
> > A possible scenario may include:
> >     ktable.toStream.filter(filterFunc).foreach( workFunc )
> >     //filterFunc true if value == (Y,1), else false
> > If the intermediate event (`A|Y,1`) is never produced + filtered, then
> > workFunc will not be executed. If I am mistaken on this point, please let
> > me know.
> >
> >
> >
> > *Monkey Wrench)If you change the foreign key (Say, A|Z,1) while the
> updates
> > in step 2 & 3 above are processing in step 4, the results will all be
> > rejected anyways upon returning to the LHS. So even if we send the
> payload,
> > the results will be rejected as stale.*
> >
> > *Conclusion:*
> > My two cents is that without full, in-order, intermediate record
> production
> > (due to the issues I illustrated in C-P1), I do not believe we should be
> > overly concerned about the other subset of intermediate messages. Fixing
> > the "duplicate" output records by sending the entire payload over the
> wire
> > is only a partial solution, as changes to theLHS FK can stomp them
> anyways
> > (see Monkey Wrench, immediately above).  If we want all intermediate
> > results to be produced, we will need to come up with another solution,
> and
> > still accept that it wont be possible to produce some (again, see C-P1).
> I
> > do not believe this is worth the effort.
> >
> > The main issue is "how do we not overwrite final results with stale
> > results?" I do not think that we can find a satisfactory
> intermediate-data
> > production mechanism with the current design. However, if we cannot use
> > offsets (see b) above) or timestamps to then I do not see any way forward
> > to ensuring consistent intermediate event production.
> >
> > Thanks
> > Adam
> >
> >
> > On Thu, Feb 28, 2019 at 9:20 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> Adam,
> >>
> >> I finally had the time to review the KIP and catch up on the mailing
> >> list discussion. Thanks a lot for putting this together! Great work! Of
> >> course also big thanks to Jan who started the KIP initially.
> >>
> >> This is a long email, because I re-read the discussion for multiple
> >> month and reply to many things... I don't think there is a need to reply
> >> to every point I mention. Just want to add my 2 cents to a couple of
> >> points that were discussed.
> >>
> >>
> >> (0) Overall the design makes sense to me. The API is intuitive and clean
> >> now. The API in the original proposal leaked a lot of implementation
> >> details, what was a major concern to me. I also believe that it's
> >> important to partition the data of the result KTable correctly (the
> >> KScatteredTable does violate this; ie, the "key is useless" as Jan
> >> phrased it), thus the last step seems to be mandatory to me. Also adding
> >> a KScatteredKTable adds a lot of new public API that is basically just
> >> duplicating existing APIs (filter, mapValue, groupBy, toStream, join).
> >> Lastly, I am happy that we don't need to "watermark/header" stuff to fix
> >> the ordering race condition.
> >>
> >> (1) About the optimization potential for multiple consecutive join: I
> >> think we could tackle this with the optimization framework we have in
> >> place now.
> >>
> >> (2) I was also thinking about left/outer join, and I believe that we
> >> could add a left-join easily (as follow up work; even if I think it's
> >> not a big addition to the current design). However, an outer-join does
> >> not make too much sense because we don't have a key for the result
> >> KTable of "right hand side" records that don't join (ie, the
> >> right-outer-join part cannot be done).
> >>
> >> (3) About the "emit on change" vs "emit on update" discussion. I think
> >> this is orthogonal to this KIP and I would stick with "emit on update"
> >> because this is the current behavior of all existing operators. If we
> >> want to change it, we should consider to do this for all operators. I
> >> also believe, even if it does not change the API, it should be backed
> >> with a KIP, because it is a semantics (ie, major) change.
> >>
> >>
> >>
> >> @Jan:
> >>
> >>> I have a lengthy track record of loosing those kinda arguments within
> >> the streams community and I have no clue why
> >>
> >> Because you are a power user, that has different goals in mind. We tend
> >> to optimize the API that it's easy to use for non-power users what is
> >> the majority of people. The KScatteredTable is a hard to grog concept...
> >>
> >>> where simplicity isn't really that as users still need to understand it
> >> I argue
> >>
> >> I disagree here. If we do a good job designing the APIs, user don't need
> >> to understand the nitty-gritty details, and it "just works".
> >>
> >>
> >> For the performance discussion, ie, which side is "larger": this does
> >> not really matter (number of keys is irrelevant) IHMO. The question is,
> >> which side is _updated_ more often and what is "n" (the factor "n" would
> >> not be relevant for Jan's proposal though). For every left hand side
> >> update, we send 2 messages to the right hand side and get 2 messages
> >> back. For every right hand side update we send n messages to the left
> >> hand side.
> >>
> >> I agree with Jan we can't know this though (not the irrelevant "size" of
> >> each side, nor the "n", nor the update rate).
> >>
> >>
> >>
> >>
> >>
> >> Finally, couple of questions/comments on the KIP (please reply to this
> >> part :)):
> >>
> >>  - For the materialized combined-key store, why do we need to disable
> >> caching? And why do we need to flush the store?
> >>
> >>  - About resolving order:
> >>
> >> (a) for each LHS update, we need to send two records to the RHS (one to
> >> "unsubscribe" the old FK, and on to "subscribe" to the new FK). The KIP
> >> further proposes to send two records back: `null` for the unsubscribe
> >> and a new join "result" for the new FK. This can result in ordering
> >> issues that we want to resolve with the FK lookup in the final step.
> >>
> >>> The thing is that in that last join, we have the opportunity to compare
> >> the
> >>> current FK in the left table with the incoming PK of the right table.
> If
> >>> they don't match, we just drop the event, since it must be outdated.
> >>
> >> Jan criticized this as "swallowing" updates if they arrive out-of-order
> >> and the delete is not reflected in the result KTable (if I understood
> >> him correctly). I disagree with Jan, and actually think, we should
> >> always avoid the delete on the result KTable to begin with:
> >>
> >> If both records arrive in the correct order on the LHS, we would still
> >> produce two output messages downstream. This is intuitive, because we
> >> _know_ that a single update to the LHS table, should result in _one_
> >> update to the result KTable. And we also know, that the update is based
> >> on (ie, correct for) the new FK.
> >>
> >> Thus, I am wondering why we would need to send the `null` message back
> >> (from RHS to LHS) in the first place?
> >>
> >> Instead, we could encode if the RHS should send something back or not.
> >> This way, an "unsubscribe" message will only update the store for the
> >> CominedKey (ie, delete the corresponding entry) and only the new FK will
> >> trigger a join lookup in the RHS table to compute a "result" that is
> >> sent back. If the LHS input is a tombstone, we send on "unsubscribe" as
> >> always, plus a `null` "subscribe" message: this ensures that we still
> >> get a join result tombstone back to update (ie, delete the entry from)
> >> the result KTable.
> >>
> >> Example: (we start with two empty tables)
> >>
> >> 1- RHS is updated to Y|foo
> >> 2- RHS is updated to Z|bar
> >> 3- LHS is updates to A|Y
> >>    -> sends Y|A+ subscribe message to RHS (no unsubscribe necessary)
> >> 4- RHS processes Y|A+ subscribe message
> >>    -> sends A|Y,foo message back
> >> 5- LHS processed A|Y,foo and produces result record A|Y,foo
> >> 6- LSH is updates to A|Z
> >>    -> sends Y|A- unsubscribe message to RHS
> >>    -> sends Z|A+ subscribe message to RHS
> >> 7- RHS processes Y|A- unsubscribe message (update store only)
> >> 8- RHS processes Z|A+ subscribe message
> >>    -> sends A|Z,bar message back
> >> 9- LHS processed A|Z,bar and produces result record A|Z,bar
> >>
> >>
> >> delete case (cont example):
> >>
> >> 10- LSH is updates to A|null (tombstone)
> >>     -> sends Z|A* subscribe message to RHS
> >>     (* indicates tombstone, we still need to encode A to be able to
> >> delete on RHS)
> >> 11- RHS processes Z|A* subscribe message (update store, ie, delete)
> >>     -> sends A|null message back
> >> 13- LHS processed A|null and produces result record A|null
> >>
> >>
> >> Maybe we could even shortcut this further, by sending only the old FK
> >> "unsubscribe" message and emit a tombstone to the result KTable
> >> directly. If there are "stuck" join results for the same LHS record on
> >> the RHS that arrive later, we can detect this case, because there is no
> >> LHS record anymore, and thus drop those records. However, I am not 100%
> >> sure if this would be correct (cf. point (c) below).
> >>
> >> delete case (optimized):
> >>
> >> 10- LSH is updates to A|null (tombstone)
> >>     -> sends Z|A- unsubscribe message to RHS
> >>     -> produces result record A|null directly
> >> 11- RHS processes Z|A- unsubscribe message (update store only)
> >>
> >>
> >> Also note that we still need the logic to resolve un-order, because
> >> there might also be un-order to consecutive LHS updates to the same
> >> record between subscribe messages, too. My proposal above only gets rid
> >> of the race condition within a single LHS update (ie, race between
> >> unsubscribe and subscribe).
> >>
> >>
> >> (b) About using "offsets" to resolve ordering issue: I don't think this
> >> would work. The join input table would be created as
> >>
> >>     stream.flatMapValues().groupByKey().aggregate()
> >>
> >> For this case, multiple KTable updates have the same input record and
> >> thus the same offset. Hence, there is no guarantee that offsets are
> >> unique and thus we cannot use them to resolve update conflicts.
> >>
> >>
> >> (c) Using the current method to avoid races, may not be correct though
> >> (or maybe the scenario below is a case of eventual consistency and not a
> >> correctness issue -- I am not sure how to judge it).
> >>
> >> We start with two empty tables:
> >>
> >> 1- RHS is updated to Y|bar
> >> 2- LHS is updated to A|Y,1
> >>    -> sends Y|A+ subscription message to RHS
> >> 3- LHS is updated to A|Y,2
> >>    -> sends Y|A- unsubscribe message to RHS
> >>    -> sends Y|A+ subscription to RHS
> >> 4- RHS processes first Y|A+ message
> >>    -> sends A|Y,bar back
> >> 5- LHS processes A|Y,bar and produces result record A|Y,2,bar
> >> 6- RHS processes Y|A- unsubscribe message (update store only)
> >> 7- RHS processes second Y|A+ subscribe message
> >>    -> sends A|Y,bar back
> >> 8- LHS processes A|Y,bar and produces result record A|Y,2,bar
> >>
> >> Thus, the first result record, that should have been `A|Y,1,bar`, is now
> >> `A|Y,2,bar`. Furthermore, we update result table from `A|Y,2,bar` to
> >> `A|Y,2,bar`.
> >>
> >> It's unclear to me, if this should be considered an incorrect
> >> (intermediate) result or not? The point being, the result is "eventually
> >> correct too early" because we join the second LHS right twice now
> >> (instead of joining each LHS record once).
> >>
> >> From a user point of view, LHS is updated with A|Y,1 and A|Y,2, and
> >> first result record does not produce any output, while second result
> >> record produces a "duplicate" result.
> >>
> >> Because all this happens on the same LHS key, I am wondering if this
> >> violated correctness (even if we end up with correct final result).
> >>
> >>
> >> Same "issue" for delete optimization as mentioned above:
> >>
> >> If we shortcut the round trip and only send one unsubscribe message and
> >> emit a tombstone directly on the LHS, there might be in-flight updates
> >> to the same LHS record "stuck" on the right hand side. If we get a new
> >> update (for the same key) to the LHS after the LHS delete, and
> >> afterwards process the "stuck" right hand side updates, we would not be
> >> able to drop those records (because the LHS table is not empty any
> >> longer). Again, we end up with the correct final result, however, I am
> >> not sure if those intermediate results should be consider "incorrect" or
> >> only "wrong in an eventual consistent way".
> >>
> >>
> >> (I hope I got all examples right... *urgs*)
> >>
> >>
> >> If you made it this far, I am very proud of you!!
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >>
> >>
> >>
> >> On 1/11/19 12:29 PM, John Roesler wrote:
> >>> Hi Jan,
> >>>
> >>> Thanks for the reply.
> >>>
> >>> It sounds like your larger point is that if we provide a building block
> >>> instead of the whole operation, then it's not too hard for users to
> >>> implement the whole operation, and maybe the building block is
> >>> independently useful.
> >>>
> >>> This is a very fair point. In fact, it's not exclusive with the current
> >>> plan,
> >>> in that we can always add the "building block" version in addition to,
> >>> rather than instead of, the full operation. It very well might be a
> >> mistake,
> >>> but I still prefer to begin by introducing the fully encapsulated
> >> operation
> >>> and subsequently consider adding the "building block" version if it
> turns
> >>> out that the encapsulated version is insufficient.
> >>>
> >>> IMHO, one of Streams's strengths over other processing frameworks
> >>> is a simple API, so simplicity as a design goal seems to suggest that:
> >>>> a.tomanyJoin(B)
> >>> is preferable to
> >>>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
> >>> at least to start with.
> >>>
> >>> To answer your question about my latter potential optimization,
> >>> no I don't have any code to look at. But, yes, the implementation
> >>> would bring B into A's tasks and keep them in a state store for
> joining.
> >>> Thanks for that reference, it does indeed sound similar to what
> >>> MapJoin does in Hive.
> >>>
> >>> Thanks again,
> >>> -John
> >>>
> >>> On Mon, Jan 7, 2019 at 5:06 PM Jan Filipiak <Ja...@trivago.com>
> >>> wrote:
> >>>
> >>>>
> >>>>
> >>>> On 02.01.2019 23:44, John Roesler wrote:
> >>>>> However, you seem to have a strong intuition that the scatter/gather
> >>>>> approach is better.
> >>>>> Is this informed by your actual applications at work? Perhaps you can
> >>>>> provide an example
> >>>>> data set and sequence of operations so we can all do the math and
> agree
> >>>>> with you.
> >>>>> It seems like we should have a convincing efficiency argument before
> >>>>> choosing a more
> >>>>> complicated API over a simpler one.
> >>>>
> >>>> The way I see this is simple. If we only provide the basic
> >>>> implementation of 1:n join (repartition by FK, Range scan on Foreign
> >>>> table update). Then this is such a fundamental building block.
> >>>>
> >>>> I do A join B.
> >>>>
> >>>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This
> >>>> pretty much performs all your "wire saving optimisations". I don't
> know!
> >>>> to be honest if someone did put this ContextAwareMapper() that was
> >>>> discussed at some point. Then I could actually do the high watermark
> >>>> thing. a.contextMap(reatain(key, fk and offset).
> >>>> omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
> >>>> I don't find the KIP though. I guess it didn't make it.
> >>>>
> >>>> After the repartition and the range read the abstraction just becomes
> to
> >>>> weak. I just showed that your implementation is my implementation with
> >>>> stuff around it.
> >>>>
> >>>> I don't know if your scatter gather thing is in code somewhere. If the
> >>>> join will only be applied after the gather phase I really wonder where
> >>>> we get the other record from? do you also persist the foreign table on
> >>>> the original side? If that is put into code somewhere already?
> >>>>
> >>>> This would essentially bring B to each of the A's tasks. Factors for
> >>>> this in my case a rather easy and dramatic. Nevertheless an approach I
> >>>> would appreciate. In Hive this could be something closely be related
> to
> >>>> the concept of a MapJoin. Something I whish we had in streams. I often
> >>>> stated that at some point we need unbounded ammount off offsets per
> >>>> topicpartition and group :D Sooooo good.
> >>>>
> >>>> Long story short. I hope you can follow my line of thought. I hope you
> >>>> can clarify my missunderstanding how the join is performed on A side
> >>>> without materializing B there.
> >>>>
> >>>> I would love if streams would get it right. The basic rule I always
> say
> >>>> is do what Hive does. done.
> >>>>
> >>>>
> >>>>>
> >>>>> Last thought:
> >>>>>> Regarding what will be observed. I consider it a plus that all
> events
> >>>>>> that are in the inputs have an respective output. Whereas your
> >> solution
> >>>>>> might "swallow" events.
> >>>>>
> >>>>> I didn't follow this. Following Adam's example, we have two join
> >>>> results: a
> >>>>> "dead" one and
> >>>>> a "live" one. If we get the dead one first, both solutions emit it,
> >>>>> followed by the live result.
> >>>>
> >>>> there might be multiple dead once in flight right? But it doesn't
> really
> >>>> matter, I never did something with the extra benefit i mentioned.
> >>>>
> >>>
> >>
> >>
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks Adam,

*> Q)
Range scans work with caching enabled, too. Thus, there is no
functional/correctness requirement to disable caching. I cannot remember
why Jan's proposal added this? It might be an implementation detail
though (maybe just remove it from the KIP? -- might be miss leading).


*> (b)
For stream.flatMap()... There is only one input record for the LHS. This
single input record, may produce multiple updates to the KTable --
however, those update records are not written into a topic (before the
update) and thus they don't have their own offsets (if you call
context.offset() for them, you will get the offset of the original input
record, ie, the same offset for all the time). Of course, on write into
the first repartition topic they will get their own offsets. However,
this information seems not to be useful, because this offsets are not
related to the input topic offset and might be from different
partitions. Thus, I don't see how reordering could be resolves using
them? However, I don't think we need offsets anyway to resolve ordering
issues, thus, it does not really matter. (I added this just as a side
remark in my response.)


*c-P1)*
*c-P2: Duplicates)*
I see your point here and agree that it might be difficult to send _all_
result updates if the RHS processors have different load and one might
lag. However, "skipping" some intermediate updates seems to be something
different that "duplicating" an update? However, I also agree that
sending the payload to the RHS might be cost prohibitive. For your
filter example, it seems that calling `KTable.toStream().filter()` would
be incorrect, but that it should be `KTable.filter()`. For the first
scenario I agree that the output record would never make it into the
result, however, for the second case, update semantics are preserved and
thus the result would still be correct (the output record won't be in
the result KTable, either, but this would be correct! -- note that
calling `KTable.toStream()` changes the semantics(!)).

Additionally, as you mentioned ("Monkey Wrench"), there are still other
scenario for which intermediate update might get swallowed. This seems
to be a general "issue" in the current design. Maybe that was the main
criticism from Jan (and it is a quite valid point). However, which
caching enabled, swallowing updates is intended behavior anyway, so I
personally don't think this is a big concern atm. (This might go back to
Jan's proposal to disable caching to avoid swallowing any updates?)

-> would be good to get input from others about the "swallowing" issue

Thus, I agree with your conclusion that sending the payload to RHS does
not resolve this issue. However, I still believe that the current design
ensures that we don't overwrite the correct final result with a stale
intermediate result. The FK check before the result data is emitted
seems to be sufficient for that (I don't think we need timestamp or
offsets to resolve this). The FK check has the only issue, that it may
produce "duplicate" updates. Or did I miss anything?

-> would be good to get input from others about this "duplicate update"
issue



-Matthias




On 3/4/19 8:29 AM, Adam Bellemare wrote:
> Hi Matthias
> 
> Thank you for the feedback! I appreciate your well thought-out questions. I
> have tried to answer and comment on everything that I know below.
> 
> 
> 
> *> Q) For the materialized combined-key store, why do we need to disable>
> caching? And why do we need to flush the store?*
> This is an artifact from Jan's implementation that I have carried along. My
> understanding (though possibly erroneous!) is that RocksDB prefix scan
> doesn't work with the cache, and ignores any data stored within it. I have
> tried to validate this but I have not succeeded, so I believe that this
> will need more investigation and testing. I will dig deeper on this and get
> back to you.
> 
> 
> 
> *> a) Thus, I am wondering why we would need to send the `null` message
> back> (from RHS to LHS) in the first place?*
> 
> We don't need to, if we follow your subsequent tombstone suggestion.
> 
> 
> 
> 
> 
> *> (b) About using "offsets" to resolve ordering issue: I don't think this>
> would work. The join input table would be created as>
> stream.flatMapValues().groupByKey().aggregate()*
> Hmmm... I am a bit fuzzy on this part. Wouldn't the LHS processor be able
> to get the highest offset and propagate that onwards to the RHS processor?
> In my original design I had a wrapper that kept track of the input offset,
> though I suspect it did not work for the above aggregation scenario.
> 
> *c-P1)*
> Regarding the consistency examples, everything you wrote is correct as far
> as I can tell in how the proposed system would behave. Rapid updates to the
> LHS will result in some of the results being discarded (in the case of
> deletes or change of FK) or doubly-produced (discussed below, after the
> following example).
> 
> It does not seem to me to be possible to avoid quashing records that are
> late arriving from the RHS. This could commonly be exhibited by two RHS
> processors that are receiving very different loads. In the example below,
> consider RHS-1 to be heavily loaded while RHS-2 is idle.
> 
> Example:
> 1- RHS-1 is updated to Y|bar
> 2- RHS-2 is updated to Z|foo
> 3- LHS is updated to A|Y
>    -> sends Y|A+ subscription message to RHS-1
> 3- LHS is updated to A|Z
>    -> sends Y|A- unsubscribe message to RHS-1
>    -> sends Z|A+ subscription to RHS-2
> 4- RHS-2 processes Z|A message immediately
>    -> sends A|Z,foo back
> 5- LHS processes A|Z,foo and produces result record A|join(Z,foo)
> 4- RHS-1 processes Y|A message
>    -> sends A|Y,bar back
> 4- RHS-1 processes Y|A- unsubscribe message
>    -> sends A|null message back
> X- LHS processes A|Y,bar, compares it to A|Z, and discards it due to
> staleness.
> X- LHS processes A|null, compares it to A|Z, and discards it due to
> staleness.
> 
> In this case, intermediate messages were discarded due to staleness. From
> the outside, this can be seen as "incorrect" because these intermediate
> results were not shown. However, there is no possible way for RHS-2 to know
> to delay production of its event until RHS-1 has completed its
> propagations. If we wish to produce all intermediate events, in order, we
> must maintain state on the LHS about which events have been sent out, await
> their return, and only publish them in order. Aside from the obvious
> complexity and memory requirements, the intermediate events would
> immediately be stomped by the final output.
> 
> 
> *c-P2: Duplicates)*
> With regards to duplicates (as per the double-sending of `A|Y,2,bar`), one
> option is to ship the entire payload of the LHS over to the RHS, and either
> join there or ship the entire payload back along with the RHS record. We
> would still need to compare the FK on the LHS to ensure that it is still
> valid. To take your example and expand it:
> 
> 1- RHS is updated to Y|bar
> 2- LHS is updated to A|Y,1
>    -> sends Y|(A, (Y,1))+ subscription message to RHS
> 3- LHS is updated to A|Y,2
>    -> sends Y|(A, (Y,1))- unsubscribe message to RHS
>    -> sends Y|(A, (Y,2))+ subscription to RHS
> 4- RHS processes first Y|A+ message
>    -> sends A|(A, (Y,1)),bar back
> 5- LHS processes A|(A, (Y,1)),bar and produces result record A|Y,1,bar
> 6- RHS processes Y|(A, (Y,1))- unsubscribe message (update store only)
> 7- RHS processes second Y|(A, (Y,2))+ subscribe message
>    -> sends A|(A, (Y,2)),bar back
> 8- LHS processes A|(A, (Y,2)),bar and produces result record A|Y,2,bar
> 
> Thus, the first result record is now `A|Y,1,bar`, while the second is
> `A|Y,2,bar`.
> 
> This will add substantially to the data payload size. The question here
> then becomes, "In which scenario is this a necessity"?
> 
> A possible scenario may include:
>     ktable.toStream.filter(filterFunc).foreach( workFunc )
>     //filterFunc true if value == (Y,1), else false
> If the intermediate event (`A|Y,1`) is never produced + filtered, then
> workFunc will not be executed. If I am mistaken on this point, please let
> me know.
> 
> 
> 
> *Monkey Wrench)If you change the foreign key (Say, A|Z,1) while the updates
> in step 2 & 3 above are processing in step 4, the results will all be
> rejected anyways upon returning to the LHS. So even if we send the payload,
> the results will be rejected as stale.*
> 
> *Conclusion:*
> My two cents is that without full, in-order, intermediate record production
> (due to the issues I illustrated in C-P1), I do not believe we should be
> overly concerned about the other subset of intermediate messages. Fixing
> the "duplicate" output records by sending the entire payload over the wire
> is only a partial solution, as changes to theLHS FK can stomp them anyways
> (see Monkey Wrench, immediately above).  If we want all intermediate
> results to be produced, we will need to come up with another solution, and
> still accept that it wont be possible to produce some (again, see C-P1). I
> do not believe this is worth the effort.
> 
> The main issue is "how do we not overwrite final results with stale
> results?" I do not think that we can find a satisfactory intermediate-data
> production mechanism with the current design. However, if we cannot use
> offsets (see b) above) or timestamps to then I do not see any way forward
> to ensuring consistent intermediate event production.
> 
> Thanks
> Adam
> 
> 
> On Thu, Feb 28, 2019 at 9:20 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> Adam,
>>
>> I finally had the time to review the KIP and catch up on the mailing
>> list discussion. Thanks a lot for putting this together! Great work! Of
>> course also big thanks to Jan who started the KIP initially.
>>
>> This is a long email, because I re-read the discussion for multiple
>> month and reply to many things... I don't think there is a need to reply
>> to every point I mention. Just want to add my 2 cents to a couple of
>> points that were discussed.
>>
>>
>> (0) Overall the design makes sense to me. The API is intuitive and clean
>> now. The API in the original proposal leaked a lot of implementation
>> details, what was a major concern to me. I also believe that it's
>> important to partition the data of the result KTable correctly (the
>> KScatteredTable does violate this; ie, the "key is useless" as Jan
>> phrased it), thus the last step seems to be mandatory to me. Also adding
>> a KScatteredKTable adds a lot of new public API that is basically just
>> duplicating existing APIs (filter, mapValue, groupBy, toStream, join).
>> Lastly, I am happy that we don't need to "watermark/header" stuff to fix
>> the ordering race condition.
>>
>> (1) About the optimization potential for multiple consecutive join: I
>> think we could tackle this with the optimization framework we have in
>> place now.
>>
>> (2) I was also thinking about left/outer join, and I believe that we
>> could add a left-join easily (as follow up work; even if I think it's
>> not a big addition to the current design). However, an outer-join does
>> not make too much sense because we don't have a key for the result
>> KTable of "right hand side" records that don't join (ie, the
>> right-outer-join part cannot be done).
>>
>> (3) About the "emit on change" vs "emit on update" discussion. I think
>> this is orthogonal to this KIP and I would stick with "emit on update"
>> because this is the current behavior of all existing operators. If we
>> want to change it, we should consider to do this for all operators. I
>> also believe, even if it does not change the API, it should be backed
>> with a KIP, because it is a semantics (ie, major) change.
>>
>>
>>
>> @Jan:
>>
>>> I have a lengthy track record of loosing those kinda arguments within
>> the streams community and I have no clue why
>>
>> Because you are a power user, that has different goals in mind. We tend
>> to optimize the API that it's easy to use for non-power users what is
>> the majority of people. The KScatteredTable is a hard to grog concept...
>>
>>> where simplicity isn't really that as users still need to understand it
>> I argue
>>
>> I disagree here. If we do a good job designing the APIs, user don't need
>> to understand the nitty-gritty details, and it "just works".
>>
>>
>> For the performance discussion, ie, which side is "larger": this does
>> not really matter (number of keys is irrelevant) IHMO. The question is,
>> which side is _updated_ more often and what is "n" (the factor "n" would
>> not be relevant for Jan's proposal though). For every left hand side
>> update, we send 2 messages to the right hand side and get 2 messages
>> back. For every right hand side update we send n messages to the left
>> hand side.
>>
>> I agree with Jan we can't know this though (not the irrelevant "size" of
>> each side, nor the "n", nor the update rate).
>>
>>
>>
>>
>>
>> Finally, couple of questions/comments on the KIP (please reply to this
>> part :)):
>>
>>  - For the materialized combined-key store, why do we need to disable
>> caching? And why do we need to flush the store?
>>
>>  - About resolving order:
>>
>> (a) for each LHS update, we need to send two records to the RHS (one to
>> "unsubscribe" the old FK, and on to "subscribe" to the new FK). The KIP
>> further proposes to send two records back: `null` for the unsubscribe
>> and a new join "result" for the new FK. This can result in ordering
>> issues that we want to resolve with the FK lookup in the final step.
>>
>>> The thing is that in that last join, we have the opportunity to compare
>> the
>>> current FK in the left table with the incoming PK of the right table. If
>>> they don't match, we just drop the event, since it must be outdated.
>>
>> Jan criticized this as "swallowing" updates if they arrive out-of-order
>> and the delete is not reflected in the result KTable (if I understood
>> him correctly). I disagree with Jan, and actually think, we should
>> always avoid the delete on the result KTable to begin with:
>>
>> If both records arrive in the correct order on the LHS, we would still
>> produce two output messages downstream. This is intuitive, because we
>> _know_ that a single update to the LHS table, should result in _one_
>> update to the result KTable. And we also know, that the update is based
>> on (ie, correct for) the new FK.
>>
>> Thus, I am wondering why we would need to send the `null` message back
>> (from RHS to LHS) in the first place?
>>
>> Instead, we could encode if the RHS should send something back or not.
>> This way, an "unsubscribe" message will only update the store for the
>> CominedKey (ie, delete the corresponding entry) and only the new FK will
>> trigger a join lookup in the RHS table to compute a "result" that is
>> sent back. If the LHS input is a tombstone, we send on "unsubscribe" as
>> always, plus a `null` "subscribe" message: this ensures that we still
>> get a join result tombstone back to update (ie, delete the entry from)
>> the result KTable.
>>
>> Example: (we start with two empty tables)
>>
>> 1- RHS is updated to Y|foo
>> 2- RHS is updated to Z|bar
>> 3- LHS is updates to A|Y
>>    -> sends Y|A+ subscribe message to RHS (no unsubscribe necessary)
>> 4- RHS processes Y|A+ subscribe message
>>    -> sends A|Y,foo message back
>> 5- LHS processed A|Y,foo and produces result record A|Y,foo
>> 6- LSH is updates to A|Z
>>    -> sends Y|A- unsubscribe message to RHS
>>    -> sends Z|A+ subscribe message to RHS
>> 7- RHS processes Y|A- unsubscribe message (update store only)
>> 8- RHS processes Z|A+ subscribe message
>>    -> sends A|Z,bar message back
>> 9- LHS processed A|Z,bar and produces result record A|Z,bar
>>
>>
>> delete case (cont example):
>>
>> 10- LSH is updates to A|null (tombstone)
>>     -> sends Z|A* subscribe message to RHS
>>     (* indicates tombstone, we still need to encode A to be able to
>> delete on RHS)
>> 11- RHS processes Z|A* subscribe message (update store, ie, delete)
>>     -> sends A|null message back
>> 13- LHS processed A|null and produces result record A|null
>>
>>
>> Maybe we could even shortcut this further, by sending only the old FK
>> "unsubscribe" message and emit a tombstone to the result KTable
>> directly. If there are "stuck" join results for the same LHS record on
>> the RHS that arrive later, we can detect this case, because there is no
>> LHS record anymore, and thus drop those records. However, I am not 100%
>> sure if this would be correct (cf. point (c) below).
>>
>> delete case (optimized):
>>
>> 10- LSH is updates to A|null (tombstone)
>>     -> sends Z|A- unsubscribe message to RHS
>>     -> produces result record A|null directly
>> 11- RHS processes Z|A- unsubscribe message (update store only)
>>
>>
>> Also note that we still need the logic to resolve un-order, because
>> there might also be un-order to consecutive LHS updates to the same
>> record between subscribe messages, too. My proposal above only gets rid
>> of the race condition within a single LHS update (ie, race between
>> unsubscribe and subscribe).
>>
>>
>> (b) About using "offsets" to resolve ordering issue: I don't think this
>> would work. The join input table would be created as
>>
>>     stream.flatMapValues().groupByKey().aggregate()
>>
>> For this case, multiple KTable updates have the same input record and
>> thus the same offset. Hence, there is no guarantee that offsets are
>> unique and thus we cannot use them to resolve update conflicts.
>>
>>
>> (c) Using the current method to avoid races, may not be correct though
>> (or maybe the scenario below is a case of eventual consistency and not a
>> correctness issue -- I am not sure how to judge it).
>>
>> We start with two empty tables:
>>
>> 1- RHS is updated to Y|bar
>> 2- LHS is updated to A|Y,1
>>    -> sends Y|A+ subscription message to RHS
>> 3- LHS is updated to A|Y,2
>>    -> sends Y|A- unsubscribe message to RHS
>>    -> sends Y|A+ subscription to RHS
>> 4- RHS processes first Y|A+ message
>>    -> sends A|Y,bar back
>> 5- LHS processes A|Y,bar and produces result record A|Y,2,bar
>> 6- RHS processes Y|A- unsubscribe message (update store only)
>> 7- RHS processes second Y|A+ subscribe message
>>    -> sends A|Y,bar back
>> 8- LHS processes A|Y,bar and produces result record A|Y,2,bar
>>
>> Thus, the first result record, that should have been `A|Y,1,bar`, is now
>> `A|Y,2,bar`. Furthermore, we update result table from `A|Y,2,bar` to
>> `A|Y,2,bar`.
>>
>> It's unclear to me, if this should be considered an incorrect
>> (intermediate) result or not? The point being, the result is "eventually
>> correct too early" because we join the second LHS right twice now
>> (instead of joining each LHS record once).
>>
>> From a user point of view, LHS is updated with A|Y,1 and A|Y,2, and
>> first result record does not produce any output, while second result
>> record produces a "duplicate" result.
>>
>> Because all this happens on the same LHS key, I am wondering if this
>> violated correctness (even if we end up with correct final result).
>>
>>
>> Same "issue" for delete optimization as mentioned above:
>>
>> If we shortcut the round trip and only send one unsubscribe message and
>> emit a tombstone directly on the LHS, there might be in-flight updates
>> to the same LHS record "stuck" on the right hand side. If we get a new
>> update (for the same key) to the LHS after the LHS delete, and
>> afterwards process the "stuck" right hand side updates, we would not be
>> able to drop those records (because the LHS table is not empty any
>> longer). Again, we end up with the correct final result, however, I am
>> not sure if those intermediate results should be consider "incorrect" or
>> only "wrong in an eventual consistent way".
>>
>>
>> (I hope I got all examples right... *urgs*)
>>
>>
>> If you made it this far, I am very proud of you!!
>>
>>
>> -Matthias
>>
>>
>>
>>
>>
>>
>> On 1/11/19 12:29 PM, John Roesler wrote:
>>> Hi Jan,
>>>
>>> Thanks for the reply.
>>>
>>> It sounds like your larger point is that if we provide a building block
>>> instead of the whole operation, then it's not too hard for users to
>>> implement the whole operation, and maybe the building block is
>>> independently useful.
>>>
>>> This is a very fair point. In fact, it's not exclusive with the current
>>> plan,
>>> in that we can always add the "building block" version in addition to,
>>> rather than instead of, the full operation. It very well might be a
>> mistake,
>>> but I still prefer to begin by introducing the fully encapsulated
>> operation
>>> and subsequently consider adding the "building block" version if it turns
>>> out that the encapsulated version is insufficient.
>>>
>>> IMHO, one of Streams's strengths over other processing frameworks
>>> is a simple API, so simplicity as a design goal seems to suggest that:
>>>> a.tomanyJoin(B)
>>> is preferable to
>>>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
>>> at least to start with.
>>>
>>> To answer your question about my latter potential optimization,
>>> no I don't have any code to look at. But, yes, the implementation
>>> would bring B into A's tasks and keep them in a state store for joining.
>>> Thanks for that reference, it does indeed sound similar to what
>>> MapJoin does in Hive.
>>>
>>> Thanks again,
>>> -John
>>>
>>> On Mon, Jan 7, 2019 at 5:06 PM Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>>
>>>>
>>>> On 02.01.2019 23:44, John Roesler wrote:
>>>>> However, you seem to have a strong intuition that the scatter/gather
>>>>> approach is better.
>>>>> Is this informed by your actual applications at work? Perhaps you can
>>>>> provide an example
>>>>> data set and sequence of operations so we can all do the math and agree
>>>>> with you.
>>>>> It seems like we should have a convincing efficiency argument before
>>>>> choosing a more
>>>>> complicated API over a simpler one.
>>>>
>>>> The way I see this is simple. If we only provide the basic
>>>> implementation of 1:n join (repartition by FK, Range scan on Foreign
>>>> table update). Then this is such a fundamental building block.
>>>>
>>>> I do A join B.
>>>>
>>>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This
>>>> pretty much performs all your "wire saving optimisations". I don't know!
>>>> to be honest if someone did put this ContextAwareMapper() that was
>>>> discussed at some point. Then I could actually do the high watermark
>>>> thing. a.contextMap(reatain(key, fk and offset).
>>>> omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
>>>> I don't find the KIP though. I guess it didn't make it.
>>>>
>>>> After the repartition and the range read the abstraction just becomes to
>>>> weak. I just showed that your implementation is my implementation with
>>>> stuff around it.
>>>>
>>>> I don't know if your scatter gather thing is in code somewhere. If the
>>>> join will only be applied after the gather phase I really wonder where
>>>> we get the other record from? do you also persist the foreign table on
>>>> the original side? If that is put into code somewhere already?
>>>>
>>>> This would essentially bring B to each of the A's tasks. Factors for
>>>> this in my case a rather easy and dramatic. Nevertheless an approach I
>>>> would appreciate. In Hive this could be something closely be related to
>>>> the concept of a MapJoin. Something I whish we had in streams. I often
>>>> stated that at some point we need unbounded ammount off offsets per
>>>> topicpartition and group :D Sooooo good.
>>>>
>>>> Long story short. I hope you can follow my line of thought. I hope you
>>>> can clarify my missunderstanding how the join is performed on A side
>>>> without materializing B there.
>>>>
>>>> I would love if streams would get it right. The basic rule I always say
>>>> is do what Hive does. done.
>>>>
>>>>
>>>>>
>>>>> Last thought:
>>>>>> Regarding what will be observed. I consider it a plus that all events
>>>>>> that are in the inputs have an respective output. Whereas your
>> solution
>>>>>> might "swallow" events.
>>>>>
>>>>> I didn't follow this. Following Adam's example, we have two join
>>>> results: a
>>>>> "dead" one and
>>>>> a "live" one. If we get the dead one first, both solutions emit it,
>>>>> followed by the live result.
>>>>
>>>> there might be multiple dead once in flight right? But it doesn't really
>>>> matter, I never did something with the extra benefit i mentioned.
>>>>
>>>
>>
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Matthias

Thank you for the feedback! I appreciate your well thought-out questions. I
have tried to answer and comment on everything that I know below.



*> Q) For the materialized combined-key store, why do we need to disable>
caching? And why do we need to flush the store?*
This is an artifact from Jan's implementation that I have carried along. My
understanding (though possibly erroneous!) is that RocksDB prefix scan
doesn't work with the cache, and ignores any data stored within it. I have
tried to validate this but I have not succeeded, so I believe that this
will need more investigation and testing. I will dig deeper on this and get
back to you.



*> a) Thus, I am wondering why we would need to send the `null` message
back> (from RHS to LHS) in the first place?*

We don't need to, if we follow your subsequent tombstone suggestion.





*> (b) About using "offsets" to resolve ordering issue: I don't think this>
would work. The join input table would be created as>
stream.flatMapValues().groupByKey().aggregate()*
Hmmm... I am a bit fuzzy on this part. Wouldn't the LHS processor be able
to get the highest offset and propagate that onwards to the RHS processor?
In my original design I had a wrapper that kept track of the input offset,
though I suspect it did not work for the above aggregation scenario.

*c-P1)*
Regarding the consistency examples, everything you wrote is correct as far
as I can tell in how the proposed system would behave. Rapid updates to the
LHS will result in some of the results being discarded (in the case of
deletes or change of FK) or doubly-produced (discussed below, after the
following example).

It does not seem to me to be possible to avoid quashing records that are
late arriving from the RHS. This could commonly be exhibited by two RHS
processors that are receiving very different loads. In the example below,
consider RHS-1 to be heavily loaded while RHS-2 is idle.

Example:
1- RHS-1 is updated to Y|bar
2- RHS-2 is updated to Z|foo
3- LHS is updated to A|Y
   -> sends Y|A+ subscription message to RHS-1
3- LHS is updated to A|Z
   -> sends Y|A- unsubscribe message to RHS-1
   -> sends Z|A+ subscription to RHS-2
4- RHS-2 processes Z|A message immediately
   -> sends A|Z,foo back
5- LHS processes A|Z,foo and produces result record A|join(Z,foo)
4- RHS-1 processes Y|A message
   -> sends A|Y,bar back
4- RHS-1 processes Y|A- unsubscribe message
   -> sends A|null message back
X- LHS processes A|Y,bar, compares it to A|Z, and discards it due to
staleness.
X- LHS processes A|null, compares it to A|Z, and discards it due to
staleness.

In this case, intermediate messages were discarded due to staleness. From
the outside, this can be seen as "incorrect" because these intermediate
results were not shown. However, there is no possible way for RHS-2 to know
to delay production of its event until RHS-1 has completed its
propagations. If we wish to produce all intermediate events, in order, we
must maintain state on the LHS about which events have been sent out, await
their return, and only publish them in order. Aside from the obvious
complexity and memory requirements, the intermediate events would
immediately be stomped by the final output.


*c-P2: Duplicates)*
With regards to duplicates (as per the double-sending of `A|Y,2,bar`), one
option is to ship the entire payload of the LHS over to the RHS, and either
join there or ship the entire payload back along with the RHS record. We
would still need to compare the FK on the LHS to ensure that it is still
valid. To take your example and expand it:

1- RHS is updated to Y|bar
2- LHS is updated to A|Y,1
   -> sends Y|(A, (Y,1))+ subscription message to RHS
3- LHS is updated to A|Y,2
   -> sends Y|(A, (Y,1))- unsubscribe message to RHS
   -> sends Y|(A, (Y,2))+ subscription to RHS
4- RHS processes first Y|A+ message
   -> sends A|(A, (Y,1)),bar back
5- LHS processes A|(A, (Y,1)),bar and produces result record A|Y,1,bar
6- RHS processes Y|(A, (Y,1))- unsubscribe message (update store only)
7- RHS processes second Y|(A, (Y,2))+ subscribe message
   -> sends A|(A, (Y,2)),bar back
8- LHS processes A|(A, (Y,2)),bar and produces result record A|Y,2,bar

Thus, the first result record is now `A|Y,1,bar`, while the second is
`A|Y,2,bar`.

This will add substantially to the data payload size. The question here
then becomes, "In which scenario is this a necessity"?

A possible scenario may include:
    ktable.toStream.filter(filterFunc).foreach( workFunc )
    //filterFunc true if value == (Y,1), else false
If the intermediate event (`A|Y,1`) is never produced + filtered, then
workFunc will not be executed. If I am mistaken on this point, please let
me know.



*Monkey Wrench)If you change the foreign key (Say, A|Z,1) while the updates
in step 2 & 3 above are processing in step 4, the results will all be
rejected anyways upon returning to the LHS. So even if we send the payload,
the results will be rejected as stale.*

*Conclusion:*
My two cents is that without full, in-order, intermediate record production
(due to the issues I illustrated in C-P1), I do not believe we should be
overly concerned about the other subset of intermediate messages. Fixing
the "duplicate" output records by sending the entire payload over the wire
is only a partial solution, as changes to theLHS FK can stomp them anyways
(see Monkey Wrench, immediately above).  If we want all intermediate
results to be produced, we will need to come up with another solution, and
still accept that it wont be possible to produce some (again, see C-P1). I
do not believe this is worth the effort.

The main issue is "how do we not overwrite final results with stale
results?" I do not think that we can find a satisfactory intermediate-data
production mechanism with the current design. However, if we cannot use
offsets (see b) above) or timestamps to then I do not see any way forward
to ensuring consistent intermediate event production.

Thanks
Adam


On Thu, Feb 28, 2019 at 9:20 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Adam,
>
> I finally had the time to review the KIP and catch up on the mailing
> list discussion. Thanks a lot for putting this together! Great work! Of
> course also big thanks to Jan who started the KIP initially.
>
> This is a long email, because I re-read the discussion for multiple
> month and reply to many things... I don't think there is a need to reply
> to every point I mention. Just want to add my 2 cents to a couple of
> points that were discussed.
>
>
> (0) Overall the design makes sense to me. The API is intuitive and clean
> now. The API in the original proposal leaked a lot of implementation
> details, what was a major concern to me. I also believe that it's
> important to partition the data of the result KTable correctly (the
> KScatteredTable does violate this; ie, the "key is useless" as Jan
> phrased it), thus the last step seems to be mandatory to me. Also adding
> a KScatteredKTable adds a lot of new public API that is basically just
> duplicating existing APIs (filter, mapValue, groupBy, toStream, join).
> Lastly, I am happy that we don't need to "watermark/header" stuff to fix
> the ordering race condition.
>
> (1) About the optimization potential for multiple consecutive join: I
> think we could tackle this with the optimization framework we have in
> place now.
>
> (2) I was also thinking about left/outer join, and I believe that we
> could add a left-join easily (as follow up work; even if I think it's
> not a big addition to the current design). However, an outer-join does
> not make too much sense because we don't have a key for the result
> KTable of "right hand side" records that don't join (ie, the
> right-outer-join part cannot be done).
>
> (3) About the "emit on change" vs "emit on update" discussion. I think
> this is orthogonal to this KIP and I would stick with "emit on update"
> because this is the current behavior of all existing operators. If we
> want to change it, we should consider to do this for all operators. I
> also believe, even if it does not change the API, it should be backed
> with a KIP, because it is a semantics (ie, major) change.
>
>
>
> @Jan:
>
> > I have a lengthy track record of loosing those kinda arguments within
> the streams community and I have no clue why
>
> Because you are a power user, that has different goals in mind. We tend
> to optimize the API that it's easy to use for non-power users what is
> the majority of people. The KScatteredTable is a hard to grog concept...
>
> > where simplicity isn't really that as users still need to understand it
> I argue
>
> I disagree here. If we do a good job designing the APIs, user don't need
> to understand the nitty-gritty details, and it "just works".
>
>
> For the performance discussion, ie, which side is "larger": this does
> not really matter (number of keys is irrelevant) IHMO. The question is,
> which side is _updated_ more often and what is "n" (the factor "n" would
> not be relevant for Jan's proposal though). For every left hand side
> update, we send 2 messages to the right hand side and get 2 messages
> back. For every right hand side update we send n messages to the left
> hand side.
>
> I agree with Jan we can't know this though (not the irrelevant "size" of
> each side, nor the "n", nor the update rate).
>
>
>
>
>
> Finally, couple of questions/comments on the KIP (please reply to this
> part :)):
>
>  - For the materialized combined-key store, why do we need to disable
> caching? And why do we need to flush the store?
>
>  - About resolving order:
>
> (a) for each LHS update, we need to send two records to the RHS (one to
> "unsubscribe" the old FK, and on to "subscribe" to the new FK). The KIP
> further proposes to send two records back: `null` for the unsubscribe
> and a new join "result" for the new FK. This can result in ordering
> issues that we want to resolve with the FK lookup in the final step.
>
> > The thing is that in that last join, we have the opportunity to compare
> the
> > current FK in the left table with the incoming PK of the right table. If
> > they don't match, we just drop the event, since it must be outdated.
>
> Jan criticized this as "swallowing" updates if they arrive out-of-order
> and the delete is not reflected in the result KTable (if I understood
> him correctly). I disagree with Jan, and actually think, we should
> always avoid the delete on the result KTable to begin with:
>
> If both records arrive in the correct order on the LHS, we would still
> produce two output messages downstream. This is intuitive, because we
> _know_ that a single update to the LHS table, should result in _one_
> update to the result KTable. And we also know, that the update is based
> on (ie, correct for) the new FK.
>
> Thus, I am wondering why we would need to send the `null` message back
> (from RHS to LHS) in the first place?
>
> Instead, we could encode if the RHS should send something back or not.
> This way, an "unsubscribe" message will only update the store for the
> CominedKey (ie, delete the corresponding entry) and only the new FK will
> trigger a join lookup in the RHS table to compute a "result" that is
> sent back. If the LHS input is a tombstone, we send on "unsubscribe" as
> always, plus a `null` "subscribe" message: this ensures that we still
> get a join result tombstone back to update (ie, delete the entry from)
> the result KTable.
>
> Example: (we start with two empty tables)
>
> 1- RHS is updated to Y|foo
> 2- RHS is updated to Z|bar
> 3- LHS is updates to A|Y
>    -> sends Y|A+ subscribe message to RHS (no unsubscribe necessary)
> 4- RHS processes Y|A+ subscribe message
>    -> sends A|Y,foo message back
> 5- LHS processed A|Y,foo and produces result record A|Y,foo
> 6- LSH is updates to A|Z
>    -> sends Y|A- unsubscribe message to RHS
>    -> sends Z|A+ subscribe message to RHS
> 7- RHS processes Y|A- unsubscribe message (update store only)
> 8- RHS processes Z|A+ subscribe message
>    -> sends A|Z,bar message back
> 9- LHS processed A|Z,bar and produces result record A|Z,bar
>
>
> delete case (cont example):
>
> 10- LSH is updates to A|null (tombstone)
>     -> sends Z|A* subscribe message to RHS
>     (* indicates tombstone, we still need to encode A to be able to
> delete on RHS)
> 11- RHS processes Z|A* subscribe message (update store, ie, delete)
>     -> sends A|null message back
> 13- LHS processed A|null and produces result record A|null
>
>
> Maybe we could even shortcut this further, by sending only the old FK
> "unsubscribe" message and emit a tombstone to the result KTable
> directly. If there are "stuck" join results for the same LHS record on
> the RHS that arrive later, we can detect this case, because there is no
> LHS record anymore, and thus drop those records. However, I am not 100%
> sure if this would be correct (cf. point (c) below).
>
> delete case (optimized):
>
> 10- LSH is updates to A|null (tombstone)
>     -> sends Z|A- unsubscribe message to RHS
>     -> produces result record A|null directly
> 11- RHS processes Z|A- unsubscribe message (update store only)
>
>
> Also note that we still need the logic to resolve un-order, because
> there might also be un-order to consecutive LHS updates to the same
> record between subscribe messages, too. My proposal above only gets rid
> of the race condition within a single LHS update (ie, race between
> unsubscribe and subscribe).
>
>
> (b) About using "offsets" to resolve ordering issue: I don't think this
> would work. The join input table would be created as
>
>     stream.flatMapValues().groupByKey().aggregate()
>
> For this case, multiple KTable updates have the same input record and
> thus the same offset. Hence, there is no guarantee that offsets are
> unique and thus we cannot use them to resolve update conflicts.
>
>
> (c) Using the current method to avoid races, may not be correct though
> (or maybe the scenario below is a case of eventual consistency and not a
> correctness issue -- I am not sure how to judge it).
>
> We start with two empty tables:
>
> 1- RHS is updated to Y|bar
> 2- LHS is updated to A|Y,1
>    -> sends Y|A+ subscription message to RHS
> 3- LHS is updated to A|Y,2
>    -> sends Y|A- unsubscribe message to RHS
>    -> sends Y|A+ subscription to RHS
> 4- RHS processes first Y|A+ message
>    -> sends A|Y,bar back
> 5- LHS processes A|Y,bar and produces result record A|Y,2,bar
> 6- RHS processes Y|A- unsubscribe message (update store only)
> 7- RHS processes second Y|A+ subscribe message
>    -> sends A|Y,bar back
> 8- LHS processes A|Y,bar and produces result record A|Y,2,bar
>
> Thus, the first result record, that should have been `A|Y,1,bar`, is now
> `A|Y,2,bar`. Furthermore, we update result table from `A|Y,2,bar` to
> `A|Y,2,bar`.
>
> It's unclear to me, if this should be considered an incorrect
> (intermediate) result or not? The point being, the result is "eventually
> correct too early" because we join the second LHS right twice now
> (instead of joining each LHS record once).
>
> From a user point of view, LHS is updated with A|Y,1 and A|Y,2, and
> first result record does not produce any output, while second result
> record produces a "duplicate" result.
>
> Because all this happens on the same LHS key, I am wondering if this
> violated correctness (even if we end up with correct final result).
>
>
> Same "issue" for delete optimization as mentioned above:
>
> If we shortcut the round trip and only send one unsubscribe message and
> emit a tombstone directly on the LHS, there might be in-flight updates
> to the same LHS record "stuck" on the right hand side. If we get a new
> update (for the same key) to the LHS after the LHS delete, and
> afterwards process the "stuck" right hand side updates, we would not be
> able to drop those records (because the LHS table is not empty any
> longer). Again, we end up with the correct final result, however, I am
> not sure if those intermediate results should be consider "incorrect" or
> only "wrong in an eventual consistent way".
>
>
> (I hope I got all examples right... *urgs*)
>
>
> If you made it this far, I am very proud of you!!
>
>
> -Matthias
>
>
>
>
>
>
> On 1/11/19 12:29 PM, John Roesler wrote:
> > Hi Jan,
> >
> > Thanks for the reply.
> >
> > It sounds like your larger point is that if we provide a building block
> > instead of the whole operation, then it's not too hard for users to
> > implement the whole operation, and maybe the building block is
> > independently useful.
> >
> > This is a very fair point. In fact, it's not exclusive with the current
> > plan,
> > in that we can always add the "building block" version in addition to,
> > rather than instead of, the full operation. It very well might be a
> mistake,
> > but I still prefer to begin by introducing the fully encapsulated
> operation
> > and subsequently consider adding the "building block" version if it turns
> > out that the encapsulated version is insufficient.
> >
> > IMHO, one of Streams's strengths over other processing frameworks
> > is a simple API, so simplicity as a design goal seems to suggest that:
> >> a.tomanyJoin(B)
> > is preferable to
> >> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
> > at least to start with.
> >
> > To answer your question about my latter potential optimization,
> > no I don't have any code to look at. But, yes, the implementation
> > would bring B into A's tasks and keep them in a state store for joining.
> > Thanks for that reference, it does indeed sound similar to what
> > MapJoin does in Hive.
> >
> > Thanks again,
> > -John
> >
> > On Mon, Jan 7, 2019 at 5:06 PM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >>
> >>
> >> On 02.01.2019 23:44, John Roesler wrote:
> >>> However, you seem to have a strong intuition that the scatter/gather
> >>> approach is better.
> >>> Is this informed by your actual applications at work? Perhaps you can
> >>> provide an example
> >>> data set and sequence of operations so we can all do the math and agree
> >>> with you.
> >>> It seems like we should have a convincing efficiency argument before
> >>> choosing a more
> >>> complicated API over a simpler one.
> >>
> >> The way I see this is simple. If we only provide the basic
> >> implementation of 1:n join (repartition by FK, Range scan on Foreign
> >> table update). Then this is such a fundamental building block.
> >>
> >> I do A join B.
> >>
> >> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This
> >> pretty much performs all your "wire saving optimisations". I don't know!
> >> to be honest if someone did put this ContextAwareMapper() that was
> >> discussed at some point. Then I could actually do the high watermark
> >> thing. a.contextMap(reatain(key, fk and offset).
> >> omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
> >> I don't find the KIP though. I guess it didn't make it.
> >>
> >> After the repartition and the range read the abstraction just becomes to
> >> weak. I just showed that your implementation is my implementation with
> >> stuff around it.
> >>
> >> I don't know if your scatter gather thing is in code somewhere. If the
> >> join will only be applied after the gather phase I really wonder where
> >> we get the other record from? do you also persist the foreign table on
> >> the original side? If that is put into code somewhere already?
> >>
> >> This would essentially bring B to each of the A's tasks. Factors for
> >> this in my case a rather easy and dramatic. Nevertheless an approach I
> >> would appreciate. In Hive this could be something closely be related to
> >> the concept of a MapJoin. Something I whish we had in streams. I often
> >> stated that at some point we need unbounded ammount off offsets per
> >> topicpartition and group :D Sooooo good.
> >>
> >> Long story short. I hope you can follow my line of thought. I hope you
> >> can clarify my missunderstanding how the join is performed on A side
> >> without materializing B there.
> >>
> >> I would love if streams would get it right. The basic rule I always say
> >> is do what Hive does. done.
> >>
> >>
> >>>
> >>> Last thought:
> >>>> Regarding what will be observed. I consider it a plus that all events
> >>>> that are in the inputs have an respective output. Whereas your
> solution
> >>>> might "swallow" events.
> >>>
> >>> I didn't follow this. Following Adam's example, we have two join
> >> results: a
> >>> "dead" one and
> >>> a "live" one. If we get the dead one first, both solutions emit it,
> >>> followed by the live result.
> >>
> >> there might be multiple dead once in flight right? But it doesn't really
> >> matter, I never did something with the extra benefit i mentioned.
> >>
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Adam,

I finally had the time to review the KIP and catch up on the mailing
list discussion. Thanks a lot for putting this together! Great work! Of
course also big thanks to Jan who started the KIP initially.

This is a long email, because I re-read the discussion for multiple
month and reply to many things... I don't think there is a need to reply
to every point I mention. Just want to add my 2 cents to a couple of
points that were discussed.


(0) Overall the design makes sense to me. The API is intuitive and clean
now. The API in the original proposal leaked a lot of implementation
details, what was a major concern to me. I also believe that it's
important to partition the data of the result KTable correctly (the
KScatteredTable does violate this; ie, the "key is useless" as Jan
phrased it), thus the last step seems to be mandatory to me. Also adding
a KScatteredKTable adds a lot of new public API that is basically just
duplicating existing APIs (filter, mapValue, groupBy, toStream, join).
Lastly, I am happy that we don't need to "watermark/header" stuff to fix
the ordering race condition.

(1) About the optimization potential for multiple consecutive join: I
think we could tackle this with the optimization framework we have in
place now.

(2) I was also thinking about left/outer join, and I believe that we
could add a left-join easily (as follow up work; even if I think it's
not a big addition to the current design). However, an outer-join does
not make too much sense because we don't have a key for the result
KTable of "right hand side" records that don't join (ie, the
right-outer-join part cannot be done).

(3) About the "emit on change" vs "emit on update" discussion. I think
this is orthogonal to this KIP and I would stick with "emit on update"
because this is the current behavior of all existing operators. If we
want to change it, we should consider to do this for all operators. I
also believe, even if it does not change the API, it should be backed
with a KIP, because it is a semantics (ie, major) change.



@Jan:

> I have a lengthy track record of loosing those kinda arguments within the streams community and I have no clue why

Because you are a power user, that has different goals in mind. We tend
to optimize the API that it's easy to use for non-power users what is
the majority of people. The KScatteredTable is a hard to grog concept...

> where simplicity isn't really that as users still need to understand it I argue

I disagree here. If we do a good job designing the APIs, user don't need
to understand the nitty-gritty details, and it "just works".


For the performance discussion, ie, which side is "larger": this does
not really matter (number of keys is irrelevant) IHMO. The question is,
which side is _updated_ more often and what is "n" (the factor "n" would
not be relevant for Jan's proposal though). For every left hand side
update, we send 2 messages to the right hand side and get 2 messages
back. For every right hand side update we send n messages to the left
hand side.

I agree with Jan we can't know this though (not the irrelevant "size" of
each side, nor the "n", nor the update rate).





Finally, couple of questions/comments on the KIP (please reply to this
part :)):

 - For the materialized combined-key store, why do we need to disable
caching? And why do we need to flush the store?

 - About resolving order:

(a) for each LHS update, we need to send two records to the RHS (one to
"unsubscribe" the old FK, and on to "subscribe" to the new FK). The KIP
further proposes to send two records back: `null` for the unsubscribe
and a new join "result" for the new FK. This can result in ordering
issues that we want to resolve with the FK lookup in the final step.

> The thing is that in that last join, we have the opportunity to compare the
> current FK in the left table with the incoming PK of the right table. If
> they don't match, we just drop the event, since it must be outdated.

Jan criticized this as "swallowing" updates if they arrive out-of-order
and the delete is not reflected in the result KTable (if I understood
him correctly). I disagree with Jan, and actually think, we should
always avoid the delete on the result KTable to begin with:

If both records arrive in the correct order on the LHS, we would still
produce two output messages downstream. This is intuitive, because we
_know_ that a single update to the LHS table, should result in _one_
update to the result KTable. And we also know, that the update is based
on (ie, correct for) the new FK.

Thus, I am wondering why we would need to send the `null` message back
(from RHS to LHS) in the first place?

Instead, we could encode if the RHS should send something back or not.
This way, an "unsubscribe" message will only update the store for the
CominedKey (ie, delete the corresponding entry) and only the new FK will
trigger a join lookup in the RHS table to compute a "result" that is
sent back. If the LHS input is a tombstone, we send on "unsubscribe" as
always, plus a `null` "subscribe" message: this ensures that we still
get a join result tombstone back to update (ie, delete the entry from)
the result KTable.

Example: (we start with two empty tables)

1- RHS is updated to Y|foo
2- RHS is updated to Z|bar
3- LHS is updates to A|Y
   -> sends Y|A+ subscribe message to RHS (no unsubscribe necessary)
4- RHS processes Y|A+ subscribe message
   -> sends A|Y,foo message back
5- LHS processed A|Y,foo and produces result record A|Y,foo
6- LSH is updates to A|Z
   -> sends Y|A- unsubscribe message to RHS
   -> sends Z|A+ subscribe message to RHS
7- RHS processes Y|A- unsubscribe message (update store only)
8- RHS processes Z|A+ subscribe message
   -> sends A|Z,bar message back
9- LHS processed A|Z,bar and produces result record A|Z,bar


delete case (cont example):

10- LSH is updates to A|null (tombstone)
    -> sends Z|A* subscribe message to RHS
    (* indicates tombstone, we still need to encode A to be able to
delete on RHS)
11- RHS processes Z|A* subscribe message (update store, ie, delete)
    -> sends A|null message back
13- LHS processed A|null and produces result record A|null


Maybe we could even shortcut this further, by sending only the old FK
"unsubscribe" message and emit a tombstone to the result KTable
directly. If there are "stuck" join results for the same LHS record on
the RHS that arrive later, we can detect this case, because there is no
LHS record anymore, and thus drop those records. However, I am not 100%
sure if this would be correct (cf. point (c) below).

delete case (optimized):

10- LSH is updates to A|null (tombstone)
    -> sends Z|A- unsubscribe message to RHS
    -> produces result record A|null directly
11- RHS processes Z|A- unsubscribe message (update store only)


Also note that we still need the logic to resolve un-order, because
there might also be un-order to consecutive LHS updates to the same
record between subscribe messages, too. My proposal above only gets rid
of the race condition within a single LHS update (ie, race between
unsubscribe and subscribe).


(b) About using "offsets" to resolve ordering issue: I don't think this
would work. The join input table would be created as

    stream.flatMapValues().groupByKey().aggregate()

For this case, multiple KTable updates have the same input record and
thus the same offset. Hence, there is no guarantee that offsets are
unique and thus we cannot use them to resolve update conflicts.


(c) Using the current method to avoid races, may not be correct though
(or maybe the scenario below is a case of eventual consistency and not a
correctness issue -- I am not sure how to judge it).

We start with two empty tables:

1- RHS is updated to Y|bar
2- LHS is updated to A|Y,1
   -> sends Y|A+ subscription message to RHS
3- LHS is updated to A|Y,2
   -> sends Y|A- unsubscribe message to RHS
   -> sends Y|A+ subscription to RHS
4- RHS processes first Y|A+ message
   -> sends A|Y,bar back
5- LHS processes A|Y,bar and produces result record A|Y,2,bar
6- RHS processes Y|A- unsubscribe message (update store only)
7- RHS processes second Y|A+ subscribe message
   -> sends A|Y,bar back
8- LHS processes A|Y,bar and produces result record A|Y,2,bar

Thus, the first result record, that should have been `A|Y,1,bar`, is now
`A|Y,2,bar`. Furthermore, we update result table from `A|Y,2,bar` to
`A|Y,2,bar`.

It's unclear to me, if this should be considered an incorrect
(intermediate) result or not? The point being, the result is "eventually
correct too early" because we join the second LHS right twice now
(instead of joining each LHS record once).

From a user point of view, LHS is updated with A|Y,1 and A|Y,2, and
first result record does not produce any output, while second result
record produces a "duplicate" result.

Because all this happens on the same LHS key, I am wondering if this
violated correctness (even if we end up with correct final result).


Same "issue" for delete optimization as mentioned above:

If we shortcut the round trip and only send one unsubscribe message and
emit a tombstone directly on the LHS, there might be in-flight updates
to the same LHS record "stuck" on the right hand side. If we get a new
update (for the same key) to the LHS after the LHS delete, and
afterwards process the "stuck" right hand side updates, we would not be
able to drop those records (because the LHS table is not empty any
longer). Again, we end up with the correct final result, however, I am
not sure if those intermediate results should be consider "incorrect" or
only "wrong in an eventual consistent way".


(I hope I got all examples right... *urgs*)


If you made it this far, I am very proud of you!!


-Matthias






On 1/11/19 12:29 PM, John Roesler wrote:
> Hi Jan,
> 
> Thanks for the reply.
> 
> It sounds like your larger point is that if we provide a building block
> instead of the whole operation, then it's not too hard for users to
> implement the whole operation, and maybe the building block is
> independently useful.
> 
> This is a very fair point. In fact, it's not exclusive with the current
> plan,
> in that we can always add the "building block" version in addition to,
> rather than instead of, the full operation. It very well might be a mistake,
> but I still prefer to begin by introducing the fully encapsulated operation
> and subsequently consider adding the "building block" version if it turns
> out that the encapsulated version is insufficient.
> 
> IMHO, one of Streams's strengths over other processing frameworks
> is a simple API, so simplicity as a design goal seems to suggest that:
>> a.tomanyJoin(B)
> is preferable to
>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
> at least to start with.
> 
> To answer your question about my latter potential optimization,
> no I don't have any code to look at. But, yes, the implementation
> would bring B into A's tasks and keep them in a state store for joining.
> Thanks for that reference, it does indeed sound similar to what
> MapJoin does in Hive.
> 
> Thanks again,
> -John
> 
> On Mon, Jan 7, 2019 at 5:06 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
> 
>>
>>
>> On 02.01.2019 23:44, John Roesler wrote:
>>> However, you seem to have a strong intuition that the scatter/gather
>>> approach is better.
>>> Is this informed by your actual applications at work? Perhaps you can
>>> provide an example
>>> data set and sequence of operations so we can all do the math and agree
>>> with you.
>>> It seems like we should have a convincing efficiency argument before
>>> choosing a more
>>> complicated API over a simpler one.
>>
>> The way I see this is simple. If we only provide the basic
>> implementation of 1:n join (repartition by FK, Range scan on Foreign
>> table update). Then this is such a fundamental building block.
>>
>> I do A join B.
>>
>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This
>> pretty much performs all your "wire saving optimisations". I don't know!
>> to be honest if someone did put this ContextAwareMapper() that was
>> discussed at some point. Then I could actually do the high watermark
>> thing. a.contextMap(reatain(key, fk and offset).
>> omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
>> I don't find the KIP though. I guess it didn't make it.
>>
>> After the repartition and the range read the abstraction just becomes to
>> weak. I just showed that your implementation is my implementation with
>> stuff around it.
>>
>> I don't know if your scatter gather thing is in code somewhere. If the
>> join will only be applied after the gather phase I really wonder where
>> we get the other record from? do you also persist the foreign table on
>> the original side? If that is put into code somewhere already?
>>
>> This would essentially bring B to each of the A's tasks. Factors for
>> this in my case a rather easy and dramatic. Nevertheless an approach I
>> would appreciate. In Hive this could be something closely be related to
>> the concept of a MapJoin. Something I whish we had in streams. I often
>> stated that at some point we need unbounded ammount off offsets per
>> topicpartition and group :D Sooooo good.
>>
>> Long story short. I hope you can follow my line of thought. I hope you
>> can clarify my missunderstanding how the join is performed on A side
>> without materializing B there.
>>
>> I would love if streams would get it right. The basic rule I always say
>> is do what Hive does. done.
>>
>>
>>>
>>> Last thought:
>>>> Regarding what will be observed. I consider it a plus that all events
>>>> that are in the inputs have an respective output. Whereas your solution
>>>> might "swallow" events.
>>>
>>> I didn't follow this. Following Adam's example, we have two join
>> results: a
>>> "dead" one and
>>> a "live" one. If we get the dead one first, both solutions emit it,
>>> followed by the live result.
>>
>> there might be multiple dead once in flight right? But it doesn't really
>> matter, I never did something with the extra benefit i mentioned.
>>
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
On 11.01.2019 21:29, John Roesler wrote:
> Hi Jan,
>
> Thanks for the reply.
>
> It sounds like your larger point is that if we provide a building block
> instead of the whole operation, then it's not too hard for users to
> implement the whole operation, and maybe the building block is
> independently useful.

exactly
>
> This is a very fair point. In fact, it's not exclusive with the current
> plan,
> in that we can always add the "building block" version in addition to,
> rather than instead of, the full operation. It very well might be a mistake,
> but I still prefer to begin by introducing the fully encapsulated operation
> and subsequently consider adding the "building block" version if it turns
> out that the encapsulated version is insufficient.

Raising my hand here, I wont be using the new API unless the scattered 
table is there. I am going to stick with my PAPI solution.

>
> IMHO, one of Streams's strengths over other processing frameworks
> is a simple API, so simplicity as a design goal seems to suggest that:
>> a.tomanyJoin(B)
> is preferable to
>> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
> at least to start with.
>
> To answer your question about my latter potential optimization,
> no I don't have any code to look at. But, yes, the implementation
> would bring B into A's tasks and keep them in a state store for joining.
> Thanks for that reference, it does indeed sound similar to what
> MapJoin does in Hive.

always a pleasure with you John.

>
> Thanks again,
> -John

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Jan,

Thanks for the reply.

It sounds like your larger point is that if we provide a building block
instead of the whole operation, then it's not too hard for users to
implement the whole operation, and maybe the building block is
independently useful.

This is a very fair point. In fact, it's not exclusive with the current
plan,
in that we can always add the "building block" version in addition to,
rather than instead of, the full operation. It very well might be a mistake,
but I still prefer to begin by introducing the fully encapsulated operation
and subsequently consider adding the "building block" version if it turns
out that the encapsulated version is insufficient.

IMHO, one of Streams's strengths over other processing frameworks
is a simple API, so simplicity as a design goal seems to suggest that:
> a.tomanyJoin(B)
is preferable to
> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A)
at least to start with.

To answer your question about my latter potential optimization,
no I don't have any code to look at. But, yes, the implementation
would bring B into A's tasks and keep them in a state store for joining.
Thanks for that reference, it does indeed sound similar to what
MapJoin does in Hive.

Thanks again,
-John

On Mon, Jan 7, 2019 at 5:06 PM Jan Filipiak <Ja...@trivago.com>
wrote:

>
>
> On 02.01.2019 23:44, John Roesler wrote:
> > However, you seem to have a strong intuition that the scatter/gather
> > approach is better.
> > Is this informed by your actual applications at work? Perhaps you can
> > provide an example
> > data set and sequence of operations so we can all do the math and agree
> > with you.
> > It seems like we should have a convincing efficiency argument before
> > choosing a more
> > complicated API over a simpler one.
>
> The way I see this is simple. If we only provide the basic
> implementation of 1:n join (repartition by FK, Range scan on Foreign
> table update). Then this is such a fundamental building block.
>
> I do A join B.
>
> a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This
> pretty much performs all your "wire saving optimisations". I don't know!
> to be honest if someone did put this ContextAwareMapper() that was
> discussed at some point. Then I could actually do the high watermark
> thing. a.contextMap(reatain(key, fk and offset).
> omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
> I don't find the KIP though. I guess it didn't make it.
>
> After the repartition and the range read the abstraction just becomes to
> weak. I just showed that your implementation is my implementation with
> stuff around it.
>
> I don't know if your scatter gather thing is in code somewhere. If the
> join will only be applied after the gather phase I really wonder where
> we get the other record from? do you also persist the foreign table on
> the original side? If that is put into code somewhere already?
>
> This would essentially bring B to each of the A's tasks. Factors for
> this in my case a rather easy and dramatic. Nevertheless an approach I
> would appreciate. In Hive this could be something closely be related to
> the concept of a MapJoin. Something I whish we had in streams. I often
> stated that at some point we need unbounded ammount off offsets per
> topicpartition and group :D Sooooo good.
>
> Long story short. I hope you can follow my line of thought. I hope you
> can clarify my missunderstanding how the join is performed on A side
> without materializing B there.
>
> I would love if streams would get it right. The basic rule I always say
> is do what Hive does. done.
>
>
> >
> > Last thought:
> >> Regarding what will be observed. I consider it a plus that all events
> >> that are in the inputs have an respective output. Whereas your solution
> >> might "swallow" events.
> >
> > I didn't follow this. Following Adam's example, we have two join
> results: a
> > "dead" one and
> > a "live" one. If we get the dead one first, both solutions emit it,
> > followed by the live result.
>
> there might be multiple dead once in flight right? But it doesn't really
> matter, I never did something with the extra benefit i mentioned.
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

On 02.01.2019 23:44, John Roesler wrote:
> However, you seem to have a strong intuition that the scatter/gather
> approach is better.
> Is this informed by your actual applications at work? Perhaps you can
> provide an example
> data set and sequence of operations so we can all do the math and agree
> with you.
> It seems like we should have a convincing efficiency argument before
> choosing a more
> complicated API over a simpler one.

The way I see this is simple. If we only provide the basic 
implementation of 1:n join (repartition by FK, Range scan on Foreign 
table update). Then this is such a fundamental building block.

I do A join B.

a.map(retain(key and FK)).tomanyJoin(B).groupBy(a.key()).join(A). This 
pretty much performs all your "wire saving optimisations". I don't know! 
to be honest if someone did put this ContextAwareMapper() that was 
discussed at some point. Then I could actually do the high watermark 
thing. a.contextMap(reatain(key, fk and offset). 
omanyJoin(B).aggregate(a.key(), oldest offset wins).join(A).
I don't find the KIP though. I guess it didn't make it.

After the repartition and the range read the abstraction just becomes to 
weak. I just showed that your implementation is my implementation with 
stuff around it.

I don't know if your scatter gather thing is in code somewhere. If the 
join will only be applied after the gather phase I really wonder where 
we get the other record from? do you also persist the foreign table on 
the original side? If that is put into code somewhere already?

This would essentially bring B to each of the A's tasks. Factors for 
this in my case a rather easy and dramatic. Nevertheless an approach I 
would appreciate. In Hive this could be something closely be related to 
the concept of a MapJoin. Something I whish we had in streams. I often 
stated that at some point we need unbounded ammount off offsets per 
topicpartition and group :D Sooooo good.

Long story short. I hope you can follow my line of thought. I hope you 
can clarify my missunderstanding how the join is performed on A side 
without materializing B there.

I would love if streams would get it right. The basic rule I always say 
is do what Hive does. done.


>
> Last thought:
>> Regarding what will be observed. I consider it a plus that all events
>> that are in the inputs have an respective output. Whereas your solution
>> might "swallow" events.
>
> I didn't follow this. Following Adam's example, we have two join results: a
> "dead" one and
> a "live" one. If we get the dead one first, both solutions emit it,
> followed by the live result.

there might be multiple dead once in flight right? But it doesn't really 
matter, I never did something with the extra benefit i mentioned.

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Just pull your version through now, I think I should shut up.
Anyhow answered inline.

Hope you had good hollidays!

best Jan

On 02.01.2019 21:35, Adam Bellemare wrote:
> Hi Jan
>
> Ahh, I got it! It is deterministic once you apply the groupBy function you
> mentioned a few months ago to the output, but not before you apply it...
> correct? I was not thinking about the groupBy function.

Its also deterministic before the group by, but the key usually is not 
too usefull.

>
> Here's how I understand how it could work from an API perspective: I am
> going to use the terminology "KScatteredTable" to represent the
> intermediate table that is not yet resolved - basically the join was
> performed but no race condition handling is done.
>
> If I wanted to join three KTables together on foreign keys, one of the ways
> I could do it is:
>
> KScatteredTable scatteredOne =  ktableOne.oneToManyJoin(kTableTwo,
> joinerFuncTwo, foreignKeyExtractorTwo);
> KScatteredTable scatteredTwo = scatteredOne.oneToManyJoin(kTableThree,
> joinerFuncThree, foreignKeyExtractorThree)
>
> //Now I groupBy the key that I want to obtain, and I can resolve the out of
> order dependencies here.
> scatteredTwo.groupBy( keyValueMapper )   ( shown here:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-SolutionB-User-ManagedGroupBy(Jan's)
> )
>
> Is this in line with what you're doing? Can this be done without exposing
> the CombinedKey? As you mentioned before "A Table
> KTable<CombinedKey<A,B>,JoinedResult> is not a good return type. It breaks
> the KTable invariant that a table is currently partitioned by its key".

The combined key has all sorts of issues, My implementation back then 
got stuck when trying to provide a built in Avro combined key serde 
because non from the schema regs guys could tell me how to have 2 keys 
per topic and it it was okay to come up with new topic names. So just 
ignore this and have them implement a serde on their own. Otherwise this 
whole crazy avro users want a say again. So not showing the CombinedKey 
in the DSL is a fair point. But I couldnt find a way to hide it.

> With that being said, are the only two operations that a KScatteredTable
> would need to support be oneToManyJoin and groupBy?

I think it is a full KTable, operations that require co-partitioning 
could arguably overriden to throw. But I do nothing else but group by 
and then the next one to many join.

>
> Thanks for your thoughts
>
> Adam
>
>
> On Wed, Jan 2, 2019 at 3:07 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Hi Adam,
>>
>> I am kinda surprised! Yes my solution of course is correct. Don't really
>> know what to show in an example as I am convinced you grabbed the
>> concept of how mine works,
>>
>> If there is a race condition there is a race condition. It doesn't
>> matter if there is 10 minutes or milliseconds between events. Either
>> they are properly guarded or not. My solution has no such race
>> condition. It is 'eventual consistent'. You gonna see all sort of stuff
>> coming up during a reprocess.
>>
>> The user can still fk it up later though. But that is usual business.
>>
>> In reality I try to supress updates from left sides as long as possible
>> because right side updates are more expensive if left is already
>> fullish. So that limits the space a little but there are no grantees.
>> The result however, after lag is zero is the same every time.
>>
>> The trade-offs can be shifted as you like. My solution gives full power
>> to the user and only does a minimum in the framework. You push
>> everything into streams.
>>
>> If you ask me, not a good choice. Will anyone listen. No.
>> I do actually think its to late to do my way. It's not like if you
>> haven't been gone through the effort and building it.
>>
>> Just wanted to give you guys another chance, to think it through  ;)
>>
>> Regarding what will be observed. I consider it a plus that all events
>> that are in the inputs have an respective output. Whereas your solution
>> might "swallow" events.
>>
>> Best Jan
>>
>>
>> On 02.01.2019 15:30, Adam Bellemare wrote:
>>> Jan
>>>
>>> I have been thinking a lot about the history of the discussion and your
>>> original proposal, and why you believe it is a better solution. The
>> biggest
>>> problem with your original proposed design is that it seems to me to be
>>> non-deterministic. It is subject to race conditions that are dependent
>>> entirely on the data, and without resolution of these races you can end
>> up
>>> with different results each time. If I am mistaken and this is indeed
>>> deterministic, then please let me know and provide an explanation,
>> ideally
>>> with an example.
>>>
>>> The way I see it is that you will get very different answers to your
>>> non-race-condition-resolved join topology, especially if you are nesting
>> it
>>> with additional joins as you have indicated you are doing. Consider
>>> rebuilding an application state from the beginning of two topics. If the
>>> left/this side has multiple foreign-key changes in a row, spaced out
>> every
>>> ten minutes, you may see something like this:
>>>
>>> (foo, foreignKey=red) t=0
>>> (foo, foreignKey=blue) t=0+10m
>>> (foo, foreignKey=green) t=0+20m
>>> (foo, foreignKey=purple) t=0+30m
>>> (foo, foreignKey=blue) t=0+40m
>>> (foo, foreignKey=white) t=0+50m
>>>
>>> During realtime processing, all of the updates may have correctly
>>> propagated because it took less than 10 minutes to resolve each join.
>> Upon
>>> rebuilding from the start, however, all of these events would be
>> processed
>>> in quick succession. The presence or absence of data will affect the
>>> results of your join, and the results can vary with each run depending on
>>> the data. Because of this, I cannot support any kind of solution that
>> would
>>> allow the exposure of an unresolved intermediate state. I can understand
>> if
>>> you don't support this, but this is why, as you said, you have the
>> freedom
>>> to use the Processor API.
>>>
>>>
>>> With that being said, either the solution that I originally proposed
>>> (join's ocurring on the foreign node) or John + Guozhang's solution
>>> (registering with the foreign node for notifications) is fine with me -
>>> both have the same API and we can evaluate it further during
>> implementation.
>>>
>>>
>>> Thanks
>>>
>>> Adam
>>>
>>> On Thu, Dec 27, 2018 at 2:38 PM Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> just want to let you guys know that this thing is spiralling out of
>>>> control if you ask me.
>>>>
>>>> First you take away the possibility for the user to optimize. Now you
>>>> pile up complexity to perform some afterwards optimisation, that from my
>>>> POV completely misses the point. As if the actual call to the joiner
>>>> really gonna be an expensive part. It wont. Truth is, you don't have a
>>>> clue which side is gonna be smaller. might be the key you shuffle around
>>>> is >>> than the value on the other side already.
>>>>
>>>> You know my opinion on this. For me its dead, I just leave you the
>>>> message here as an opportunity to reconsider the choices that were made.
>>>>
>>>> Whish y'll a happy new year :)
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On 27.12.2018 17:22, Adam Bellemare wrote:
>>>>> Hi All
>>>>>
>>>>> Sorry for the delay - holidays and all. I have since updated the KIP
>> with
>>>>> John's original suggestion and have pruned a number of the no longer
>>>>> relevant diagrams. Any more comments would be welcomed, otherwise I
>> will
>>>>> look to kick off the vote again shortly.
>>>>>
>>>>> Thanks
>>>>> Adam
>>>>>
>>>>> On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <
>> adam.bellemare@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>>> Hi John and Guozhang
>>>>>>
>>>>>> Ah yes, I lost that in the mix! Thanks for the convergent solutions -
>> I
>>>> do
>>>>>> think that the attachment that John included makes for a better
>> design.
>>>> It
>>>>>> should also help with overall performance as very high-cardinality
>>>> foreign
>>>>>> keyed data (say millions of events with the same entity) will be able
>> to
>>>>>> leverage the multiple nodes for join functionality instead of having
>> it
>>>> all
>>>>>> performed in one node. There is still a bottleneck in the right table
>>>>>> having to propagate all those events, but with slimmer structures,
>> less
>>>> IO
>>>>>> and no need to perform the join I think the throughput will be much
>>>> higher
>>>>>> in those scenarios.
>>>>>>
>>>>>> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
>>>>>> version of John's diagram and ensure that the example flow matches
>>>>>> correctly. Then I can go back to working on the PR to match the
>> diagram.
>>>>>>
>>>>>> Thanks both of you for all the help - very much appreciated.
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com>
>>>> wrote:
>>>>>>
>>>>>>> Hi John,
>>>>>>>
>>>>>>> Just made a pass on your diagram (nice hand-drawing btw!), and
>>>> obviously
>>>>>>> we
>>>>>>> are thinking about the same thing :) A neat difference that I like,
>> is
>>>>>>> that
>>>>>>> in the pre-join repartition topic we can still send message in the
>>>> format
>>>>>>> of `K=k, V=(i=2)` while using "i" as the partition key in
>>>>>>> StreamsPartition,
>>>>>>> this way we do not need to even augment the key for the repartition
>>>> topic,
>>>>>>> but just do a projection on the foreign key part but trim all other
>>>>>>> fields:
>>>>>>> as long as we still materialize the store as `A-2` co-located with
>> the
>>>>>>> right KTable, that is fine.
>>>>>>>
>>>>>>> As I mentioned in my previous email, I also think this has a few
>>>>>>> advantages
>>>>>>> on saving over-the-wire bytes as well as disk bytes.
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io>
>>>> wrote:
>>>>>>>
>>>>>>>> Hi Guozhang,
>>>>>>>>
>>>>>>>> Thanks for taking a look! I think Adam's already addressed your
>>>>>>> questions
>>>>>>>> as well as I could have.
>>>>>>>>
>>>>>>>> Hi Adam,
>>>>>>>>
>>>>>>>> Thanks for updating the KIP. It looks great, especially how all the
>>>>>>>> need-to-know information is right at the top, followed by the
>> details.
>>>>>>>>
>>>>>>>> Also, thanks for that high-level diagram. Actually, now that I'm
>>>> looking
>>>>>>>> at it, I think part of my proposal got lost in translation,
>> although I
>>>>>>> do
>>>>>>>> think that what you have there is also correct.
>>>>>>>>
>>>>>>>> I sketched up a crude diagram based on yours and attached it to the
>>>> KIP
>>>>>>>> (I'm not sure if attached or inline images work on the mailing
>> list):
>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
>>>>>>>> . It's also attached to this email for convenience.
>>>>>>>>
>>>>>>>> Hopefully, you can see how it's intended to line up, and which parts
>>>> are
>>>>>>>> modified.
>>>>>>>> At a high level, instead of performing the join on the right-hand
>>>> side,
>>>>>>>> we're essentially just registering interest, like "LHS key A wishes
>> to
>>>>>>>> receive updates for RHS key 2". Then, when there is a new "interest"
>>>> or
>>>>>>> any
>>>>>>>> updates to the RHS records, it "broadcasts" its state back to the
>> LHS
>>>>>>>> records who are interested in it.
>>>>>>>>
>>>>>>>> Thus, instead of sending the LHS values to the RHS joiner workers
>> and
>>>>>>> then
>>>>>>>> sending the join results back to the LHS worke be co-partitioned and
>>>>>>>> validated, we instead only send the LHS *keys* to the RHS workers
>> and
>>>>>>> then
>>>>>>>> only the RHS k/v back to be joined by the LHS worker.
>>>>>>>>
>>>>>>>> I've been considering both your diagram and mine, and I *think* what
>>>> I'm
>>>>>>>> proposing has a few advantages.
>>>>>>>>
>>>>>>>> Here are some points of interest as you look at the diagram:
>>>>>>>> * When we extract the foreign key and send it to the Pre-Join
>>>>>>> Repartition
>>>>>>>> Topic, we can send only the FK/PK pair. There's no need to worry
>> about
>>>>>>>> custom partitioner logic, since we can just use the foreign key
>>>> plainly
>>>>>>> as
>>>>>>>> the repartition record key. Also, we save on transmitting the LHS
>>>> value,
>>>>>>>> since we only send its key in this step.
>>>>>>>> * We also only need to store the RHSKey:LHSKey mapping in the
>>>>>>>> MaterializedSubscriptionStore, saving on disk. We can use the same
>>>> rocks
>>>>>>>> key format you proposed and the same algorithm involving range scans
>>>>>>> when
>>>>>>>> the RHS records get updated.
>>>>>>>> * Instead of joining on the right side, all we do is compose a
>>>>>>>> re-repartition record so we can broadcast the RHS k/v pair back to
>> the
>>>>>>>> original LHS partition. (this is what the "rekey" node is doing)
>>>>>>>> * Then, there is a special kind of Joiner that's co-resident in the
>>>> same
>>>>>>>> StreamTask as the LHS table, subscribed to the Post-Join Repartition
>>>>>>> Topic.
>>>>>>>> ** This Joiner is *not* triggered directly by any changes in the LHS
>>>>>>>> KTable. Instead, LHS events indirectly trigger the join via the
>> whole
>>>>>>>> lifecycle.
>>>>>>>> ** For each event arriving from the Post-Join Repartition Topic, the
>>>>>>>> Joiner looks up the corresponding record in the LHS KTable. It
>>>> validates
>>>>>>>> the FK as you noted, discarding any inconsistent events. Otherwise,
>> it
>>>>>>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the
>>>> join
>>>>>>>> result
>>>>>>>> ** Note that the Joiner itself is stateless, so materializing the
>> join
>>>>>>>> result is optional, just as with the 1:1 joins.
>>>>>>>>
>>>>>>>> So in summary:
>>>>>>>> * instead of transmitting the LHS keys and values to the right and
>> the
>>>>>>>> JoinResult back to the left, we only transmit the LHS keys to the
>>>> right
>>>>>>> and
>>>>>>>> the RHS values to the left. Assuming the average RHS value is on
>>>> smaller
>>>>>>>> than or equal to the average join result size, it's a clear win on
>>>>>>> broker
>>>>>>>> traffic. I think this is actually a reasonable assumption, which we
>>>> can
>>>>>>>> discuss more if you're suspicious.
>>>>>>>> * we only need one copy of the data (the left and right tables need
>> to
>>>>>>> be
>>>>>>>> materialized) and one extra copy of the PK:FK pairs in the
>>>> Materialized
>>>>>>>> Subscription Store. Materializing the join result is optional, just
>> as
>>>>>>> with
>>>>>>>> the existing 1:1 joins.
>>>>>>>> * we still need the fancy range-scan algorithm on the right to
>> locate
>>>>>>> all
>>>>>>>> interested LHS keys when a RHS value is updated, but we don't need a
>>>>>>> custom
>>>>>>>> partitioner for either repartition topic (this is of course a
>>>>>>> modification
>>>>>>>> we could make to your version as well)
>>>>>>>>
>>>>>>>> How does this sound to you? (And did I miss anything?)
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
>>>>>>> adam.bellemare@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi John & Guozhang
>>>>>>>>>
>>>>>>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up
>> the
>>>>>>> KIP,
>>>>>>>>> pruned much of what I wrote and put a simplified diagram near the
>> top
>>>>>>> to
>>>>>>>>> illustrate the workflow. I encapsulated Jan's content at the bottom
>>>> of
>>>>>>> the
>>>>>>>>> document. I believe it is simpler to read by far now.
>>>>>>>>>
>>>>>>>>> @Guozhang Wang <wa...@gmail.com>:
>>>>>>>>>> #1: rekey left table
>>>>>>>>>>      -> source from the left upstream, send to rekey-processor to
>>>>>>> generate
>>>>>>>>> combined key, and then sink to copartition topic.
>>>>>>>>> Correct.
>>>>>>>>>
>>>>>>>>>> #2: first-join with right table
>>>>>>>>>>      -> source from the right table upstream, materialize the right
>>>>>>> table.
>>>>>>>>>>      -> source from the co-partition topic, materialize the rekeyed
>>>> left
>>>>>>>>> table, join with the right table, rekey back, and then sink to the
>>>>>>>>> rekeyed-back topic.
>>>>>>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I need
>>>> the
>>>>>>>>> Foreign-Key value generated in #1 above to compare in the
>> resolution
>>>>>>>>> stage.
>>>>>>>>>
>>>>>>>>>> #3: second join
>>>>>>>>>>       -> source from the rekeyed-back topic, materialize the
>> rekeyed
>>>>>>> back
>>>>>>>>> table.
>>>>>>>>>>      -> source from the left upstream, materialize the left table,
>>>> join
>>>>>>>>> with
>>>>>>>>> the rekeyed back table.
>>>>>>>>> Almost - As each event comes in, we just run it through a stateful
>>>>>>>>> processor that checks the original ("This") KTable for the key. The
>>>>>>> value
>>>>>>>>> payload then has the foreignKeyExtractor applied again as in Part
>> #1
>>>>>>>>> above,
>>>>>>>>> and gets the current foreign key. Then we compare it to the joined
>>>>>>> event
>>>>>>>>> that we are currently resolving. If they have the same foreign-key,
>>>>>>>>> propagate the result out. If they don't, throw the event away.
>>>>>>>>>
>>>>>>>>> The end result is that we do need to materialize 2 additional
>> tables
>>>>>>>>> (left/this-combinedkey table, and the final Joined table) as I've
>>>>>>>>> illustrated in the updated KIP. I hope the diagram clears it up a
>> lot
>>>>>>>>> better. Please let me know.
>>>>>>>>>
>>>>>>>>> Thanks again
>>>>>>>>> Adam
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> John,
>>>>>>>>>>
>>>>>>>>>> Thanks a lot for the suggestions on refactoring the wiki, I agree
>>>>>>> with
>>>>>>>>> you
>>>>>>>>>> that we should consider the KIP proposal to be easily understood
>> by
>>>>>>>>> anyone
>>>>>>>>>> in the future to read, and hence should provide a good summary on
>>>> the
>>>>>>>>>> user-facing interfaces, as well as rejected alternatives to
>>>> represent
>>>>>>>>>> briefly "how we came a long way to this conclusion, and what we
>> have
>>>>>>>>>> argued, disagreed, and agreed about, etc" so that readers do not
>>>>>>> need to
>>>>>>>>>> dig into the DISCUSS thread to get all the details. We can, of
>>>>>>> course,
>>>>>>>>> keep
>>>>>>>>>> the implementation details like "workflows" on the wiki page as a
>>>>>>>>> addendum
>>>>>>>>>> section since it also has correlations.
>>>>>>>>>>
>>>>>>>>>> Regarding your proposal on comment 6): that's a very interesting
>>>>>>> idea!
>>>>>>>>> Just
>>>>>>>>>> to clarify that I understands it fully correctly: the proposal's
>>>>>>>>> resulted
>>>>>>>>>> topology is still the same as the current proposal, where we will
>>>>>>> have 3
>>>>>>>>>> sub-topologies for this operator:
>>>>>>>>>>
>>>>>>>>>> #1: rekey left table
>>>>>>>>>>       -> source from the left upstream, send to rekey-processor to
>>>>>>> generate
>>>>>>>>>> combined key, and then sink to copartition topic.
>>>>>>>>>>
>>>>>>>>>> #2: first-join with right table
>>>>>>>>>>       -> source from the right table upstream, materialize the
>> right
>>>>>>> table.
>>>>>>>>>>       -> source from the co-partition topic, materialize the
>> rekeyed
>>>>>>> left
>>>>>>>>>> table, join with the right table, rekey back, and then sink to the
>>>>>>>>>> rekeyed-back topic.
>>>>>>>>>>
>>>>>>>>>> #3: second join
>>>>>>>>>>       -> source from the rekeyed-back topic, materialize the
>> rekeyed
>>>>>>> back
>>>>>>>>>> table.
>>>>>>>>>>       -> source from the left upstream, materialize the left table,
>>>> join
>>>>>>>>> with
>>>>>>>>>> the rekeyed back table.
>>>>>>>>>>
>>>>>>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology
>> since
>>>>>>>>> both of
>>>>>>>>>> them read from the left table source stream. In this workflow, we
>>>>>>> need
>>>>>>>>> to
>>>>>>>>>> materialize 4 tables (left table in #3, right table in #2, rekeyed
>>>>>>> left
>>>>>>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
>>>>>>>>>> (copartition topic, rekeyed-back topic).
>>>>>>>>>>
>>>>>>>>>> Compared with Adam's current proposal in the workflow overview, it
>>>>>>> has
>>>>>>>>> the
>>>>>>>>>> same num.materialize tables (left table, rekeyed left table, right
>>>>>>>>> table,
>>>>>>>>>> out-of-ordering resolver table), and same num.internal topics
>> (two).
>>>>>>> The
>>>>>>>>>> advantage is that on the copartition topic, we can save bandwidth
>> by
>>>>>>> not
>>>>>>>>>> sending value, and in #2 the rekeyed left table is smaller since
>> we
>>>>>>> do
>>>>>>>>> not
>>>>>>>>>> have any values to materialize. Is that right?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>
>>>>>>>>>>> Given that the committers are all pretty busy right now, I think
>>>>>>> that
>>>>>>>>> it
>>>>>>>>>>> would help if you were to refactor the KIP a little to reduce the
>>>>>>>>>> workload
>>>>>>>>>>> for reviewers.
>>>>>>>>>>>
>>>>>>>>>>> I'd recommend the following changes:
>>>>>>>>>>> * relocate all internal details to a section at the end called
>>>>>>>>> something
>>>>>>>>>>> like "Implementation Notes" or something like that.
>>>>>>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
>>>>>>> mention
>>>>>>>>>> only
>>>>>>>>>>> publicly-facing API changes.
>>>>>>>>>>> ** for example, the interface that you've already listed there,
>> as
>>>>>>>>> well
>>>>>>>>>> as
>>>>>>>>>>> a textual description of the guarantees we'll be providing (join
>>>>>>>>> result
>>>>>>>>>> is
>>>>>>>>>>> copartitioned with the LHS, and the join result is guaranteed
>>>>>>> correct)
>>>>>>>>>>>
>>>>>>>>>>> A good target would be that the whole main body of the KIP,
>>>>>>> including
>>>>>>>>>>> Status, Motivation, Proposal, Justification, and Rejected
>>>>>>> Alternatives
>>>>>>>>>> all
>>>>>>>>>>> fit "above the fold" (i.e., all fit on the screen at a
>> comfortable
>>>>>>>>> zoom
>>>>>>>>>>> level).
>>>>>>>>>>> I think the only real Rejected Alternative that bears mention at
>>>>>>> this
>>>>>>>>>> point
>>>>>>>>>>> is KScatteredTable, which you could just include the executive
>>>>>>>>> summary on
>>>>>>>>>>> (no implementation details), and link to extra details in the
>>>>>>>>>>> Implementation Notes section.
>>>>>>>>>>>
>>>>>>>>>>> Taking a look at the wiki page, ~90% of the text there is
>> internal
>>>>>>>>>> detail,
>>>>>>>>>>> which is useful for the dubious, but doesn't need to be ratified
>>>>>>> in a
>>>>>>>>>> vote
>>>>>>>>>>> (and would be subject to change without notice in the future
>>>>>>> anyway).
>>>>>>>>>>> There's also a lot of conflicting discussion, as you've very
>>>>>>>>> respectfully
>>>>>>>>>>> tried to preserve the original proposal from Jan while adding
>> your
>>>>>>>>> own.
>>>>>>>>>>> Isolating all this information in a dedicated section at the
>> bottom
>>>>>>>>> frees
>>>>>>>>>>> the voters up to focus on the public API part of the proposal,
>>>>>>> which
>>>>>>>>> is
>>>>>>>>>>> really all they need to consider.
>>>>>>>>>>>
>>>>>>>>>>> Plus, it'll be clear to future readers which parts of the
>> document
>>>>>>> are
>>>>>>>>>>> enduring, and which parts are a snapshot of our implementation
>>>>>>>>> thinking
>>>>>>>>>> at
>>>>>>>>>>> the time.
>>>>>>>>>>>
>>>>>>>>>>> I'm suggesting this because I suspect that the others haven't
>> made
>>>>>>>>> time
>>>>>>>>>> to
>>>>>>>>>>> review it partly because it seems daunting. If it seems like it
>>>>>>> would
>>>>>>>>> be
>>>>>>>>>> a
>>>>>>>>>>> huge time investment to review, people will just keep putting it
>>>>>>> off.
>>>>>>>>> But
>>>>>>>>>>> if the KIP is a single page, then they'll be more inclined to
>> give
>>>>>>> it
>>>>>>>>> a
>>>>>>>>>>> read.
>>>>>>>>>>>
>>>>>>>>>>> Honestly, I don't think the KIP itself is that controversial
>> (apart
>>>>>>>>> from
>>>>>>>>>>> the scattered table thing (sorry, Jan) ). Most of the discussion
>>>>>>> has
>>>>>>>>> been
>>>>>>>>>>> around the implementation, which we can continue more effectively
>>>>>>> in
>>>>>>>>> a PR
>>>>>>>>>>> once the KIP has passed.
>>>>>>>>>>>
>>>>>>>>>>> How does that sound?
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
>>>>>>>>> adam.bellemare@gmail.com
>>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> 1) I believe that the resolution mechanism John has proposed is
>>>>>>>>>>> sufficient
>>>>>>>>>>>> - it is clean and easy and doesn't require additional RocksDB
>>>>>>>>> stores,
>>>>>>>>>>> which
>>>>>>>>>>>> reduces the footprint greatly. I don't think we need to resolve
>>>>>>>>> based
>>>>>>>>>> on
>>>>>>>>>>>> timestamp or offset anymore, but if we decide to do to that
>>>>>>> would be
>>>>>>>>>>> within
>>>>>>>>>>>> the bounds of the existing API.
>>>>>>>>>>>>
>>>>>>>>>>>> 2) Is the current API sufficient, or does it need to be altered
>>>>>>> to
>>>>>>>>> go
>>>>>>>>>>> back
>>>>>>>>>>>> to vote?
>>>>>>>>>>>>
>>>>>>>>>>>> 3) KScatteredTable implementation can always be added in a
>> future
>>>>>>>>>>> revision.
>>>>>>>>>>>> This API does not rule it out. This implementation of this
>>>>>>> function
>>>>>>>>>> would
>>>>>>>>>>>> simply be replaced with `KScatteredTable.resolve()` while still
>>>>>>>>>>> maintaining
>>>>>>>>>>>> the existing API, thereby giving both features as Jan outlined
>>>>>>>>> earlier.
>>>>>>>>>>>> Would this work?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks Guozhang, John and Jan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <
>> john@confluent.io
>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi, all,
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> In fact, we
>>>>>>>>>>>>>>> can just keep a single final-result store with timestamps
>>>>>>> and
>>>>>>>>>> reject
>>>>>>>>>>>>> values
>>>>>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>>>>>> offset
>>>>>>>>>>> of
>>>>>>>>>>>>>> the original message.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for this point, Jan.
>>>>>>>>>>>>>
>>>>>>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
>>>>>>> the
>>>>>>>>> k/v
>>>>>>>>>>>>> store,
>>>>>>>>>>>>> as well as providing a storage-format upgrade path.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I might have missed it, but I think we have yet to discuss
>>>>>>> whether
>>>>>>>>>> it's
>>>>>>>>>>>>> safe
>>>>>>>>>>>>> or desirable just to swap topic-ordering our for
>>>>>>>>> timestamp-ordering.
>>>>>>>>>>> This
>>>>>>>>>>>>> is
>>>>>>>>>>>>> a very deep topic, and I think it would only pollute the
>>>>>>> current
>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>
>>>>>>>>>>>>> What Adam has proposed is safe, given the *current* ordering
>>>>>>>>>> semantics
>>>>>>>>>>>>> of the system. If we can agree on his proposal, I think we can
>>>>>>>>> merge
>>>>>>>>>>> the
>>>>>>>>>>>>> feature well before the conversation about timestamp ordering
>>>>>>> even
>>>>>>>>>>> takes
>>>>>>>>>>>>> place, much less reaches a conclusion. In the mean time, it
>>>>>>> would
>>>>>>>>>> seem
>>>>>>>>>>> to
>>>>>>>>>>>>> be unfortunate to have one join operator with different
>>>>>>> ordering
>>>>>>>>>>>> semantics
>>>>>>>>>>>>> from every other KTable operator.
>>>>>>>>>>>>>
>>>>>>>>>>>>> If and when that timestamp discussion takes place, many (all?)
>>>>>>>>> KTable
>>>>>>>>>>>>> operations
>>>>>>>>>>>>> will need to be updated, rendering the many:one join a small
>>>>>>>>> marginal
>>>>>>>>>>>> cost.
>>>>>>>>>>>>>
>>>>>>>>>>>>> And, just to plug it again, I proposed an algorithm above that
>>>>>>> I
>>>>>>>>>>> believe
>>>>>>>>>>>>> provides
>>>>>>>>>>>>> correct ordering without any additional metadata, and
>>>>>>> regardless
>>>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>> ordering semantics. I didn't bring it up further, because I
>>>>>>> felt
>>>>>>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>>>> only needs
>>>>>>>>>>>>> to agree on the public API, and we can discuss the
>>>>>>> implementation
>>>>>>>>> at
>>>>>>>>>>>>> leisure in
>>>>>>>>>>>>> a PR...
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
>>>>>>>>>>>>>>> Hello Adam / Jan / John,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Sorry for being late on this thread! I've finally got some
>>>>>>>>> time
>>>>>>>>>>> this
>>>>>>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
>>>>>>> I've
>>>>>>>>>> also
>>>>>>>>>>>>>> realized
>>>>>>>>>>>>>>> there are a bunch of other things I need to enqueue while
>>>>>>>>>> cleaning
>>>>>>>>>>>> them
>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>> --- sth I need to improve on my side). So here are my
>>>>>>>>> thoughts:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regarding the APIs: I like the current written API in the
>>>>>>> KIP.
>>>>>>>>>> More
>>>>>>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
>>>>>>>>>>> functionalities
>>>>>>>>>>>> as
>>>>>>>>>>>>>>> well as 2) other join types than inner as separate KIPs
>>>>>>> since
>>>>>>>>> 1)
>>>>>>>>>>> may
>>>>>>>>>>>>>> worth
>>>>>>>>>>>>>>> a general API refactoring that can benefit not only
>>>>>>> foreignkey
>>>>>>>>>>> joins
>>>>>>>>>>>>> but
>>>>>>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
>>>>>>>>>>>>>> ),
>>>>>>>>>>>>>>> and I'm not sure if other join types would actually be
>>>>>>> needed
>>>>>>>>>>> (maybe
>>>>>>>>>>>>> left
>>>>>>>>>>>>>>> join still makes sense), so it's better to
>>>>>>>>>>>>> wait-for-people-to-ask-and-add
>>>>>>>>>>>>>>> than add-sth-that-no-one-uses.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
>>>>>>> a
>>>>>>>>>>>>>>> KScatteredTable for users to inject their own optimization:
>>>>>>>>> I'd
>>>>>>>>>>>> prefer
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> do the current option as-is, and my main rationale is for
>>>>>>>>>>>> optimization
>>>>>>>>>>>>>>> rooms inside the Streams internals and the API
>>>>>>> succinctness.
>>>>>>>>> For
>>>>>>>>>>>>> advanced
>>>>>>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
>>>>>>> own
>>>>>>>>>>>>>> optimization,
>>>>>>>>>>>>>>> while it is too much of the work to use Processor API
>>>>>>>>> directly, I
>>>>>>>>>>>> think
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> can still extend the current API to support it in the
>>>>>>> future
>>>>>>>>> if
>>>>>>>>>> it
>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> no internal optimization potential. it's a myth
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ¯\_(ツ)_/¯
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> :-)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
>>>>>>> as
>>>>>>>>> I
>>>>>>>>>>>>> mentioned
>>>>>>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
>>>>>>> key-value
>>>>>>>>>>> store)
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> can actually make this step simpler than the current
>>>>>>>>> proposal. In
>>>>>>>>>>>> fact,
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> can just keep a single final-result store with timestamps
>>>>>>> and
>>>>>>>>>>> reject
>>>>>>>>>>>>>> values
>>>>>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>>>>>> offset
>>>>>>>>>>> of
>>>>>>>>>>>>>> the original message.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
>>>>>>>>> Adam
>>>>>>>>>> to
>>>>>>>>>>>>> make
>>>>>>>>>>>>>>> such HUGE progress on this KIP!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If they don't find the time:
>>>>>>>>>>>>>>>> They usually take the opposite path from me :D
>>>>>>>>>>>>>>>> so the answer would be clear.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> hence my suggestion to vote.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>> Hi Guozhang and Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
>>>>>>> KIP
>>>>>>>>>> to a
>>>>>>>>>>>>> point
>>>>>>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
>>>>>>> a
>>>>>>>>>>>>> tie-breaker,
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> you will). If you have anyone else you may think should
>>>>>>>>> look at
>>>>>>>>>>>> this,
>>>>>>>>>>>>>>>>> please tag them accordingly.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The scenario is as such:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Current Option:
>>>>>>>>>>>>>>>>> API:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>>>>>>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
>>>>>>>>>>> partition
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> foreignKey (repartition 1)
>>>>>>>>>>>>>>>>> 2) Join the data
>>>>>>>>>>>>>>>>> 3) Shuffle the data back to the original node
>>>>>>> (repartition
>>>>>>>>> 2)
>>>>>>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
>>>>>>>>>>> foreign-key
>>>>>>>>>>>>>>>> changes.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Alternate Option:
>>>>>>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
>>>>>>>>>>>>>>>>> - It would be keyed on a wrapped key function:
>>>>>>>>> <CombinedKey<KO,
>>>>>>>>>>> K>,
>>>>>>>>>>>>> VR>
>>>>>>>>>>>>>>>> (KO
>>>>>>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
>>>>>>> Result)
>>>>>>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
>>>>>>>>>>> otherwise a
>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>> would be able to perform additional functions directly
>>>>>>> from
>>>>>>>>> the
>>>>>>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
>>>>>>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
>>>>>>>>> tradeoffs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
>>>>>>> possible,
>>>>>>>>>> but
>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> require for implementation details to be made in the API
>>>>>>> and
>>>>>>>>>> some
>>>>>>>>>>>>>>>> exposure
>>>>>>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I appreciate any insight into this.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
>>>>>>>>> summary
>>>>>>>>>> is
>>>>>>>>>>>>>>>> accurate
>>>>>>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
>>>>>>> that
>>>>>>>>>>> there
>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>> risk
>>>>>>>>>>>>>>>>>> of inconsistent final states without performing the
>>>>>>>>>> resolution.
>>>>>>>>>>>> This
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> major concern for me as most of the data I have dealt
>>>>>>> with
>>>>>>>>> is
>>>>>>>>>>>>> produced
>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>> relational databases. We have seen a number of cases
>>>>>>> where
>>>>>>>>> a
>>>>>>>>>>> user
>>>>>>>>>>>> in
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
>>>>>>>>> they
>>>>>>>>>>> made
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> mistake, and then updated the field again with a new
>>>>>>> key.
>>>>>>>>> The
>>>>>>>>>>>> events
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> propagated out as they are produced, and as such we have
>>>>>>>>> had
>>>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>> cases where these inconsistencies were propagated
>>>>>>>>> downstream
>>>>>>>>>> as
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>> values due to the race conditions in the fanout of the
>>>>>>>>> data.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This solution that I propose values correctness of the
>>>>>>>>> final
>>>>>>>>>>>> result
>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>>>> other factors.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We could always move this function over to using a
>>>>>>>>>>> KScatteredTable
>>>>>>>>>>>>>>>>>> implementation in the future, and simply deprecate it
>>>>>>> this
>>>>>>>>>> join
>>>>>>>>>>>> API
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> time. I think I would like to hear more from some of the
>>>>>>>>> other
>>>>>>>>>>>> major
>>>>>>>>>>>>>>>>>> committers on which course of action they would think is
>>>>>>>>> best
>>>>>>>>>>>> before
>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>> more coding is done.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks again
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
>>>>>>>>>> john@confluent.io>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Jan and Adam,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
>>>>>>> are
>>>>>>>>>>>>> encouraging.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
>>>>>>>>> agree
>>>>>>>>>>> that
>>>>>>>>>>>>>>>> avoiding
>>>>>>>>>>>>>>>>>>> unnecessary join outputs is especially important when
>>>>>>> the
>>>>>>>>>>> fan-out
>>>>>>>>>>>>> is
>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>> high. I suppose this could also be built into the
>>>>>>>>>>> implementation
>>>>>>>>>>>>>> we're
>>>>>>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
>>>>>>>>> KIP
>>>>>>>>>>>> (since
>>>>>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>>>> API-transparent optimization).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
>>>>>>>>> didn't
>>>>>>>>>>>> bring
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
>>>>>>> the
>>>>>>>>>> KIP
>>>>>>>>>>>>> as-is,
>>>>>>>>>>>>>>>>>>> despite the disagreement.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
>>>>>>>>>>>> reasonable.
>>>>>>>>>>>>>>>>>>> It sounds like Jan values more the potential for
>>>>>>>>> developers
>>>>>>>>>> to
>>>>>>>>>>>>>> optimize
>>>>>>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
>>>>>>> whereas
>>>>>>>>>> Adam
>>>>>>>>>>>>>> places
>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> value on having a single operator that people can use
>>>>>>>>> without
>>>>>>>>>>>> extra
>>>>>>>>>>>>>>>> steps
>>>>>>>>>>>>>>>>>>> at the end.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Personally, although I do find it exceptionally
>>>>>>> annoying
>>>>>>>>>> when a
>>>>>>>>>>>>>>>> framework
>>>>>>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
>>>>>>> it
>>>>>>>>>> seems
>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>>> for a single operation.
>>>>>>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
>>>>>>>>> significant
>>>>>>>>>>>>>> latitude
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the implementation (for example, joining only at the
>>>>>>> end,
>>>>>>>>> not
>>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> middle
>>>>>>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
>>>>>>> resolution;
>>>>>>>>> how
>>>>>>>>>> we
>>>>>>>>>>>>>>>> represent
>>>>>>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
>>>>>>>>> vectors),
>>>>>>>>>>>>> etc.).
>>>>>>>>>>>>>>>> If we
>>>>>>>>>>>>>>>>>>> publish something like a KScatteredTable with the
>>>>>>>>>>>> right-partitioned
>>>>>>>>>>>>>>>> joined
>>>>>>>>>>>>>>>>>>> data, then the API pretty much locks in the
>>>>>>>>> implementation as
>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
>>>>>>> mean
>>>>>>>>>>> "seems";
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>> anyone
>>>>>>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
>>>>>>>>>>> simpler,
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
>>>>>>>>> algebra
>>>>>>>>>>>>>>>> perspective,
>>>>>>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
>>>>>>>>> KTable
>>>>>>>>>>> in
>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>>>> * That said, there might still be room in the API for a
>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
>>>>>>> then
>>>>>>>>> do
>>>>>>>>>>>> things
>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
>>>>>>> haven't
>>>>>>>>>>> thought
>>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>> all the consequences yet.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This is all just my opinion after thinking over the
>>>>>>>>>> discussion
>>>>>>>>>>> so
>>>>>>>>>>>>>>>> far...
>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
>>>>>>> the
>>>>>>>>>>>>>> prefixScan.
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> have attached the file, but I will also include the
>>>>>>> text
>>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>> body
>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
>>>>>>>>>> attached
>>>>>>>>>>>>>> files).
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
>>>>>>> large
>>>>>>>>>>> values
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
>>>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
>>>>>>>>> For
>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
>>>>>>> and a
>>>>>>>>>>> further
>>>>>>>>>>>>>>>>>>> ~35-85mS
>>>>>>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
>>>>>>>>> them.
>>>>>>>>>>>>> Iterating
>>>>>>>>>>>>>>>>>>>> through the records just to generate a simple count
>>>>>>>>> takes ~
>>>>>>>>>> 40
>>>>>>>>>>>>> times
>>>>>>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>>>>>>> than the flush + scan combined.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>>>> Setup:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
>>>>>>>>>>> (Xmx512m,
>>>>>>>>>>>>>>>> Xms512m)
>>>>>>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
>>>>>>> directly-accessible
>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
>>>>>>> the
>>>>>>>>>>>>>>>>>>>> ProcessorContext).
>>>>>>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
>>>>>>> values
>>>>>>>>>>>> provided
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> 2.1/trunk release.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> keysize = 128 bytes
>>>>>>>>>>>>>>>>>>>> valsize = 512 bytes
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Step 1:
>>>>>>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
>>>>>>>>>> left-padded
>>>>>>>>>>>>>>>>>>>> auto-incrementing integer)
>>>>>>>>>>>>>>>>>>>> Step 2:
>>>>>>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
>>>>>>>>>>>>>>>> auto-incrementing
>>>>>>>>>>>>>>>>>>>> integer)
>>>>>>>>>>>>>>>>>>>> Step 3:
>>>>>>>>>>>>>>>>>>>> Perform flush
>>>>>>>>>>>>>>>>>>>> Step 4:
>>>>>>>>>>>>>>>>>>>> Perform prefixScan
>>>>>>>>>>>>>>>>>>>> Step 5:
>>>>>>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
>>>>>>> count of
>>>>>>>>>>>> expected
>>>>>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>>>> Results:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>>>> X = 1k (11k events total)
>>>>>>>>>>>>>>>>>>>> Flush Time = 39 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>>>>>> 6.9 MB disk
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>>> X = 10k (110k events total)
>>>>>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 8 mS
>>>>>>>>>>>>>>>>>>>> 127 MB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>>> X = 100k (1.1M events total)
>>>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>>>> Flush Time = 60 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 12 mS
>>>>>>>>>>>>>>>>>>>> 678 MB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>>>>>> 576 MB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>>> X = 1MB (11M events total)
>>>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>>>> Flush Time = 52 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 19 mS
>>>>>>>>>>>>>>>>>>>> 7.2 GB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>>>> Flush Time = 84 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 34 mS
>>>>>>>>>>>>>>>>>>>> 9.1 GB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
>>>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>>>> Flush Time = 82 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 63 mS
>>>>>>>>>>>>>>>>>>>> 17GB - 276 sst files
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>>>> Flush Time = 116 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 35 mS
>>>>>>>>>>>>>>>>>>>> 23GB - 361 sst files
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Test3:
>>>>>>>>>>>>>>>>>>>> Flush Time = 103 mS
>>>>>>>>>>>>>>>>>>>> Scan Time = 82 mS
>>>>>>>>>>>>>>>>>>>> 19 GB - 300 sst files
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
>>>>>>>>> events. I
>>>>>>>>>>>> tried
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
>>>>>>> the
>>>>>>>>>> 100GB+
>>>>>>>>>>>>> range
>>>>>>>>>>>>>>>>>>> and my
>>>>>>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
>>>>>>> be
>>>>>>>>> done
>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>> suspect
>>>>>>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
>>>>>>>>>> results
>>>>>>>>>>>>>> above.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> At this point in time, I think the only major
>>>>>>> discussion
>>>>>>>>>> point
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
>>>>>>> repartitioning
>>>>>>>>>> back +
>>>>>>>>>>>>>>>> resolving
>>>>>>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
>>>>>>> the
>>>>>>>>>> client
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks folks,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
>>>>>>> the
>>>>>>>>> 2.1
>>>>>>>>>>>>> release
>>>>>>>>>>>>>>>>>>>>> caused an
>>>>>>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
>>>>>>>>> radar
>>>>>>>>>>>> (which
>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
>>>>>>>>> some
>>>>>>>>>>>> extend
>>>>>>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
>>>>>>>>> keep) me
>>>>>>>>>>>>>>>>>>> preoccupied.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
>>>>>>> guess
>>>>>>>>> Jan
>>>>>>>>>>> was
>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>> track!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
>>>>>>> and
>>>>>>>>> the
>>>>>>>>>>>> prior
>>>>>>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> API Thoughts:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
>>>>>>>>>> many-to-one
>>>>>>>>>>>>> join.
>>>>>>>>>>>>>>>>>>> Could
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
>>>>>>>>> flip
>>>>>>>>>>> the
>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
>>>>>>> join
>>>>>>>>>>> type,
>>>>>>>>>>>>> and
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
>>>>>>>>>>> one-to-one
>>>>>>>>>>>>>> join.
>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>> would work, of course, but it would be super
>>>>>>>>> inefficient
>>>>>>>>>>>>> compared
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> simple rekey-and-join.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
>>>>>>>>>> specified
>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
>>>>>>>>> join, as
>>>>>>>>>>>>>>>>>>> (neglecting
>>>>>>>>>>>>>>>>>>>>> IQ),
>>>>>>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
>>>>>>>>> it in
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> ValueJoiner.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
>>>>>>> quite
>>>>>>>>>>> right.
>>>>>>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
>>>>>>>>>>> paradigms
>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> play in
>>>>>>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
>>>>>>>>> three
>>>>>>>>>>>>>> Serialized
>>>>>>>>>>>>>>>>>>>>> args, I
>>>>>>>>>>>>>>>>>>>>>> think it would be better to have one that allows
>>>>>>>>>>> (optionally)
>>>>>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>>>>>> the 4
>>>>>>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
>>>>>>>>>>>>> Materialized.
>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
>>>>>>>>> already
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
>>>>>>>>> config
>>>>>>>>>>>> might
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> applicable.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
>>>>>>> joins
>>>>>>>>>>> don't
>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
>>>>>>>>> since
>>>>>>>>>>> the
>>>>>>>>>>>>>> rekey
>>>>>>>>>>>>>>>>>>>>>> operation needs to produce results that are
>>>>>>>>> co-partitioned
>>>>>>>>>>>> with
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> "other"
>>>>>>>>>>>>>>>>>>>>>> KTable.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
>>>>>>>>> didn't
>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
>>>>>>>>>>>> "deserializing"
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
>>>>>>>>> think
>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
>>>>>>>>> namespace
>>>>>>>>>> at
>>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>> nodes
>>>>>>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
>>>>>>>>> practically
>>>>>>>>>> no
>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
>>>>>>> API
>>>>>>>>>> change
>>>>>>>>>>>> wrt
>>>>>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
>>>>>>> placed
>>>>>>>>> on
>>>>>>>>>>>> users'
>>>>>>>>>>>>>>>>>>>>> headers.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
>>>>>>> the
>>>>>>>>>>> header
>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> all...
>>>>>>>>>>>>>>>>>>>>>> stay tuned)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
>>>>>>>>> table
>>>>>>>>>>>>> growing
>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
>>>>>>>>>>> implementing
>>>>>>>>>>>>> OCC
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
>>>>>>> the
>>>>>>>>>>> rekey
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
>>>>>>> changes.
>>>>>>>>> As
>>>>>>>>>>> such,
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
>>>>>>> offset in
>>>>>>>>>> the
>>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
>>>>>>> the
>>>>>>>>> same
>>>>>>>>>>>>> number
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> keys
>>>>>>>>>>>>>>>>>>>>>> as the source table at all times.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
>>>>>>>>> might
>>>>>>>>>> be
>>>>>>>>>>>>>> relevant
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
>>>>>>>>> timestamp
>>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
>>>>>>> the
>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>> OCC
>>>>>>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
>>>>>>>>> want to
>>>>>>>>>>> use
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
>>>>>>> use a
>>>>>>>>>>> similar
>>>>>>>>>>>>>>>>>>>>> mechanism to
>>>>>>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
>>>>>>>>>> re-keyed
>>>>>>>>>>>>>> values,
>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>> you could avoid a separate table.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
>>>>>>> long
>>>>>>>>>>> time,
>>>>>>>>>>>> so
>>>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
>>>>>>> if we
>>>>>>>>>> can
>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> HWM
>>>>>>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
>>>>>>> final
>>>>>>>>>> join
>>>>>>>>>>>>>>>>>>> instead...
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
>>>>>>> Letter
>>>>>>>>>> FK,
>>>>>>>>>>>>> (other
>>>>>>>>>>>>>>>>>>>>> data))
>>>>>>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>>>>>> 1: (A, xyz)
>>>>>>>>>>>>>>>>>>>>>> 2: (B, asd)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Right table:
>>>>>>>>>>>>>>>>>>>>>> A: EntityA
>>>>>>>>>>>>>>>>>>>>>> B: EntityB
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
>>>>>>>>> key,
>>>>>>>>>> but
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> propagating the value at all..
>>>>>>>>>>>>>>>>>>>>>> Rekey table:
>>>>>>>>>>>>>>>>>>>>>> A-1: (dummy value)
>>>>>>>>>>>>>>>>>>>>>> B-2: (dummy value)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
>>>>>>>>>>>>>>>>>>>>>> A-1: EntityA
>>>>>>>>>>>>>>>>>>>>>> B-2: EntityB
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Which gets rekeyed back:
>>>>>>>>>>>>>>>>>>>>>> 1: A, EntityA
>>>>>>>>>>>>>>>>>>>>>> 2: B, EntityB
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> And finally we do the actual join:
>>>>>>>>>>>>>>>>>>>>>> Result table:
>>>>>>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
>>>>>>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
>>>>>>>>>> opportunity
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> compare
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
>>>>>>>>> the
>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>> table. If
>>>>>>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
>>>>>>>>> must be
>>>>>>>>>>>>>> outdated.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
>>>>>>> xyz)
>>>>>>>>> gets
>>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> (1:
>>>>>>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
>>>>>>> whether
>>>>>>>>> the
>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
>>>>>>>>>> algorithm
>>>>>>>>>>>>> above,
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
>>>>>>>>> (B,
>>>>>>>>>>> xyz),
>>>>>>>>>>>>> (B,
>>>>>>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
>>>>>>>>>>> information
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
>>>>>>>>> mentioned
>>>>>>>>>>>> often
>>>>>>>>>>>>> I
>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
>>>>>>>>>>> automatically.
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>> this is the real performance eater ;)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
>>>>>>>>>>>> performance
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> range scans when records change in the right table.
>>>>>>>>> You've
>>>>>>>>>>>> said
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> you've
>>>>>>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
>>>>>>>>> for a
>>>>>>>>>>>>> while.
>>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
>>>>>>>>> you've
>>>>>>>>>>>>>> observed?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
>>>>>>>>> topmost
>>>>>>>>>>>> thing
>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
>>>>>>>>>>> performance
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
>>>>>>>>> 500k -
>>>>>>>>>>> 1kk
>>>>>>>>>>>>> rows
>>>>>>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
>>>>>>> the
>>>>>>>>> work
>>>>>>>>>>> you
>>>>>>>>>>>>>> gotta
>>>>>>>>>>>>>>>>>>>>> do, also when you pack your data into different
>>>>>>> formats,
>>>>>>>>>>>> usually
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
>>>>>>> data
>>>>>>>>> and
>>>>>>>>>>> we
>>>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>>>> really change that. It is more important for users to
>>>>>>>>>> prevent
>>>>>>>>>>>>>> useless
>>>>>>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
>>>>>>> to
>>>>>>>>> drop
>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> are not going to change my join output.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> usually it's:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
>>>>>>>>>> old.equals(new)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
>>>>>>> for
>>>>>>>>>>>> smaller
>>>>>>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
>>>>>>> then
>>>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
>>>>>>>>> sure if
>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
>>>>>>> the
>>>>>>>>>>>> original
>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>> as I
>>>>>>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
>>>>>>> keys in
>>>>>>>>>> the
>>>>>>>>>>>>> value:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>>>>>> 1: A,...
>>>>>>>>>>>>>>>>>>>>>> 2: B,...
>>>>>>>>>>>>>>>>>>>>>> 3: A,...
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Gets re-keyed:
>>>>>>>>>>>>>>>>>>>>>> A: [1, 3]
>>>>>>>>>>>>>>>>>>>>>> B: [2]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
>>>>>>>>> regular
>>>>>>>>>>>>>> single-key
>>>>>>>>>>>>>>>>>>>>> lookup.
>>>>>>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
>>>>>>>>>> values,
>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> there's
>>>>>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
>>>>>>> reference
>>>>>>>>> rhs
>>>>>>>>>>>>> records.
>>>>>>>>>>>>>>>>>>>>> Offhand,
>>>>>>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
>>>>>>>>> past
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> threshold, we
>>>>>>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
>>>>>>> cases,
>>>>>>>>> it
>>>>>>>>>>> would
>>>>>>>>>>>>> be
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
>>>>>>> be
>>>>>>>>> one
>>>>>>>>>>> per
>>>>>>>>>>>>>> (max
>>>>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
>>>>>>>>> there's
>>>>>>>>>>> some
>>>>>>>>>>>>>> extra
>>>>>>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
>>>>>>>>>> retraction.
>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
>>>>>>> just
>>>>>>>>> have
>>>>>>>>>>> (A:
>>>>>>>>>>>>>> [3]).
>>>>>>>>>>>>>>>>>>>>> Some
>>>>>>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
>>>>>>>>> for 1
>>>>>>>>>>> can
>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> retracted.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> That's all!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
>>>>>>> thoughtful
>>>>>>>>>> KIP.
>>>>>>>>>>>>> Sorry
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> discussion has been slow.
>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
>>>>>>> up,
>>>>>>>>> it
>>>>>>>>>>> just
>>>>>>>>>>>>>> goes
>>>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>>>> to discuss.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> would not expect to much attention with another
>>>>>>>>> another
>>>>>>>>>>> email
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> best Jan
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Hello Contributors
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
>>>>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>> this to
>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
>>>>>>>>> through
>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>> contributor
>>>>>>>>>>>>>>>>>>>>>>>> feedback is given.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Main points that need addressing:
>>>>>>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
>>>>>>>>>>> singular
>>>>>>>>>>>>>> graph
>>>>>>>>>>>>>>>>>>> node
>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
>>>>>>> right
>>>>>>>>>>> now. I
>>>>>>>>>>>>> am
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>> fuzzy
>>>>>>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
>>>>>>> I
>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>> help on this aspect.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
>>>>>>>>> thread
>>>>>>>>>>> has
>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>> discourse
>>>>>>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
>>>>>>> mark
>>>>>>>>>>>> proposal
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
>>>>>>> need
>>>>>>>>> to
>>>>>>>>>>>>> strictly
>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
>>>>>>> up
>>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> consumer.
>>>>>>>>>>>>>>>>>>>>> Any
>>>>>>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
>>>>>>> this
>>>>>>>>>> from
>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>>>>>>> vote?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>>>>>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
>>>>>>>>>>>>>>>>>>> Stores.persistentWindowStore,
>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
>>>>>>> segments
>>>>>>>>>> you
>>>>>>>>>>>> want
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>> large
>>>>>>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
>>>>>>>>> happens
>>>>>>>>>>> is
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
>>>>>>> data
>>>>>>>>> with
>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>> timestamps
>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
>>>>>>> of
>>>>>>>>> the
>>>>>>>>>>>> work
>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
>>>>>>> the
>>>>>>>>>>>> hopping
>>>>>>>>>>>>>>>>>>> windowed
>>>>>>>>>>>>>>>>>>>>>>> store.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
>>>>>>>>> to be
>>>>>>>>>>> two
>>>>>>>>>>>>> 24h
>>>>>>>>>>>>>>>>>>>>>>> segments.
>>>>>>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
>>>>>>> will
>>>>>>>>> be
>>>>>>>>>>>>> inserted
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>> T in
>>>>>>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
>>>>>>> overlap
>>>>>>>>> by
>>>>>>>>>>> 12h.
>>>>>>>>>>>>> As
>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>> goes on
>>>>>>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
>>>>>>> the
>>>>>>>>>>> oldest
>>>>>>>>>>>>>>>> segment
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
>>>>>>> The
>>>>>>>>>>>> records
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
>>>>>>> it
>>>>>>>>> is
>>>>>>>>>>> the
>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
>>>>>>> are
>>>>>>>>>>>> retained
>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
>>>>>>> TTL
>>>>>>>>>> at a
>>>>>>>>>>>>>>>>>>>>> segment-level,
>>>>>>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
>>>>>>>>>>>>>> ClassCastException
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>> similar.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
>>>>>>>>>>> fetch/put
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> iterate
>>>>>>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
>>>>>>>>> stream-time
>>>>>>>>>>>> under
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> hoods
>>>>>>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
>>>>>>> all
>>>>>>>>> the
>>>>>>>>>>>>> windows
>>>>>>>>>>>>>>>>>>>>> earlier?
>>>>>>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
>>>>>>>>> time
>>>>>>>>>>>>>> gurantees.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Best
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
>>>>>>>>> changed
>>>>>>>>>>> the
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
>>>>>>>>>> something
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extremely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
>>>>>>> said, I
>>>>>>>>>> will
>>>>>>>>>>>> not
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>>>>>>>>>> ahead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
>>>>>>> reshuffle/groupBy
>>>>>>>>>>> solution
>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>> propose.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
>>>>>>>>>> yourself
>>>>>>>>>>>> off
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>> current PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
>>>>>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>> happy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
>>>>>>>>>> stands
>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>>> I do
>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
>>>>>>>>> alternatives
>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
>>>>>>> which
>>>>>>>>>> they
>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> prefer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
>>>>>>> gave
>>>>>>>>> up
>>>>>>>>>> on
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
>>>>>>> pulled
>>>>>>>>>> this
>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>>>>>>>>> already.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
>>>>>>> based on
>>>>>>>>>>> PAPI.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
>>>>>>>>> windowed
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
>>>>>>> then I
>>>>>>>>>> will
>>>>>>>>>>>> come
>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
>>>>>>> any of
>>>>>>>>>> the
>>>>>>>>>>>>>> binding
>>>>>>>>>>>>>>>>>>>>> voters
>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
>>>>>>>>> updated
>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> according
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
>>>>>>> windowed
>>>>>>>>>>> store.
>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
>>>>>>> they
>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> completed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
>>>>>>>>>> already
>>>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> PR?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
>>>>>>> Missing
>>>>>>>>>>>>> something?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
>>>>>>> is
>>>>>>>>> the
>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>> link,
>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
>>>>>>>>> part of
>>>>>>>>>>>>> KIP-258
>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
>>>>>>>>>> KTable"
>>>>>>>>>>>> such
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
>>>>>>> materialized
>>>>>>>>>> store,
>>>>>>>>>>>>> i.e.
>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
>>>>>>> older
>>>>>>>>>> than
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>> key's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
>>>>>>> ordering.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
>>>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
>>>>>>>>> final
>>>>>>>>>>> step
>>>>>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
>>>>>>>>> with
>>>>>>>>>> a
>>>>>>>>>>>>> window
>>>>>>>>>>>>>>>>>>>>> store),
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
>>>>>>>>> help:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
>>>>>>>>> key-value
>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
>>>>>>>>> usage,
>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
>>>>>>>>>> ,
>>>>>>>>>>> is
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
>>>>>>> its
>>>>>>>>>>>>> timestamp
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> smaller
>>>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
>>>>>>> I
>>>>>>>>>> think
>>>>>>>>>>> it
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
>>>>>>> based
>>>>>>>>>>>>> filtering,
>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
>>>>>>> the
>>>>>>>>>>> joining
>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
>>>>>>>>> final
>>>>>>>>>>>> stage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
>>>>>>> and
>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>> non-windowed
>>>>>>>>>>>>>>>>>>>>>>>>>> KTables
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
>>>>>>>>> really
>>>>>>>>>>> have
>>>>>>>>>>>> a
>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
>>>>>>>>>>> KTable-KTable
>>>>>>>>>>>>>>>>>>> non-key
>>>>>>>>>>>>>>>>>>>>>>> joins
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
>>>>>>> Filipiak
>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
>>>>>>> would
>>>>>>>>>> grow
>>>>>>>>>>>>>>>> endlessly
>>>>>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
>>>>>>> pair
>>>>>>>>> of
>>>>>>>>>>>> (<this
>>>>>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
>>>>>>> is
>>>>>>>>> used
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> differentiate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
>>>>>>>>> proposal
>>>>>>>>>>>> would
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> replace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
>>>>>>>>> This
>>>>>>>>>>> would
>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
>>>>>>> time.
>>>>>>>>> This
>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
>>>>>>>>>> should
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> customizable
>>>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
>>>>>>>>> perhaps
>>>>>>>>>>> just
>>>>>>>>>>>>> 10
>>>>>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
>>>>>>> do
>>>>>>>>> the
>>>>>>>>>>>> trick
>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>> Even
>>>>>>>>>>>>>>>>>>>>>>>>>> if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
>>>>>>>>>>> repartitioning
>>>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
>>>>>>>>> am a
>>>>>>>>>>>> little
>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>> sceptical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
>>>>>>> one
>>>>>>>>>>> could
>>>>>>>>>>>>> run
>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
>>>>>>>>> border. I
>>>>>>>>>>> will
>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
>>>>>>>>> problematic, we
>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
>>>>>>> performance
>>>>>>>>>>> impact I
>>>>>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
>>>>>>> correct
>>>>>>>>> as
>>>>>>>>>>> is. I
>>>>>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
>>>>>>>>>>> timestamp(A)  <
>>>>>>>>>>>>>>>>>>>>>>> timestamp(B).
>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
>>>>>>> -
>>>>>>>>>> thanks
>>>>>>>>>>>> for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> diagram, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
>>>>>>> do
>>>>>>>>> not
>>>>>>>>>>> have
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
>>>>>>>>>>> available
>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
>>>>>>> Map.
>>>>>>>>>> That
>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
>>>>>>> for
>>>>>>>>>>> clarity
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> everyone
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
>>>>>>> really
>>>>>>>>> hard
>>>>>>>>>>>> work.
>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
>>>>>>>>>>> original
>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
>>>>>>> PAPI
>>>>>>>>>> and
>>>>>>>>>>>>>>>> basically
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
>>>>>>> missed
>>>>>>>>>> that
>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
>>>>>>>>> up on
>>>>>>>>>>> my
>>>>>>>>>>>>> end.
>>>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>>>>>> finish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
>>>>>>>>> week.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
>>>>>>> the
>>>>>>>>> Map
>>>>>>>>>>> stay
>>>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> State
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
>>>>>>> changes
>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>> propagated?
>>>>>>>>>>>>>>>>>>>>>>> Isn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
>>>>>>>>> state
>>>>>>>>>>>> store?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
>>>>>>>>> substractor
>>>>>>>>>> is
>>>>>>>>>>>>> gonna
>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>> `null`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
>>>>>>>>> there
>>>>>>>>>> is
>>>>>>>>>>>>> going
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
>>>>>>> this
>>>>>>>>>> store
>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
>>>>>>> is a
>>>>>>>>>>>> regular
>>>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>>>> satisfying
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
>>>>>>>>> join.
>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>> Windowed
>>>>>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
>>>>>>> statefull
>>>>>>>>>>>> operation
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
>>>>>>>>> have
>>>>>>>>>> the
>>>>>>>>>>>>>> window
>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
>>>>>>> custom
>>>>>>>>>> group
>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
>>>>>>> key i
>>>>>>>>>> think
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
>>>>>>> the
>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
>>>>>>>>> foundation
>>>>>>>>>>> to
>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
>>>>>>> user.
>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>> unfortunate. I
>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
>>>>>>> do
>>>>>>>>> not
>>>>>>>>>>>> think
>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
>>>>>>> Prajakta
>>>>>>>>>>> Dumbre <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              please remove me from this
>>>>>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              On Tue, Sep 11, 2018 at 1:29 PM
>>>>>>>>> Jan
>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <Jan.Filipiak@trivago.com
>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Hi Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > give me some time, will make
>>>>>>>>> such a
>>>>>>>>>>>>> chart.
>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>>> time i
>>>>>>>>>>>>>>>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              get along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > well with giphy and ruined
>>>>>>> all
>>>>>>>>> your
>>>>>>>>>>>>> charts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Hopefully i can get it done
>>>>>>>>> today
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > On 08.09.2018 16:00, Adam
>>>>>>>>> Bellemare
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > I have included a diagram
>>>>>>> of
>>>>>>>>>> what I
>>>>>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > I attempted this back at
>>>>>>> the
>>>>>>>>>> start
>>>>>>>>>>> of
>>>>>>>>>>>>> my
>>>>>>>>>>>>>> own
>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > solution, and since I could
>>>>>>>>> not
>>>>>>>>>> get
>>>>>>>>>>>> it
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> work I
>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              discarded the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > code. At this point in
>>>>>>> time,
>>>>>>>>> if
>>>>>>>>>> you
>>>>>>>>>>>>> wish
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              for your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > groupBy solution, I ask
>>>>>>> that
>>>>>>>>> you
>>>>>>>>>>>> please
>>>>>>>>>>>>>>>>>>> create a
>>>>>>>>>>>>>>>>>>>>>>>>>> diagram on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > carefully explaining your
>>>>>>>>>> solution.
>>>>>>>>>>>>>> Please
>>>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>>>>>> free
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the image I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > just posted as a starting
>>>>>>>>> point.
>>>>>>>>>> I
>>>>>>>>>>> am
>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>>>>>> trouble
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              understanding your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > explanations but I think
>>>>>>> that
>>>>>>>>> a
>>>>>>>>>>>>> carefully
>>>>>>>>>>>>>>>>>>>>> constructed
>>>>>>>>>>>>>>>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              will clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > any misunderstandings.
>>>>>>>>>> Alternately,
>>>>>>>>>>>>>> please
>>>>>>>>>>>>>>>>>>> post a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              comprehensive PR with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > your solution. I can only
>>>>>>>>> guess
>>>>>>>>>> at
>>>>>>>>>>>> what
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> mean, and
>>>>>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              value my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > own
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > time as much as you value
>>>>>>>>> yours,
>>>>>>>>>> I
>>>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              responsibility to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > provide an implementation
>>>>>>>>> instead
>>>>>>>>>>> of
>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > On Sat, Sep 8, 2018 at 8:00
>>>>>>>>> AM,
>>>>>>>>>> Jan
>>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <Jan.Filipiak@trivago.com
>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > > wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> Hi James,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> nice to see you beeing
>>>>>>>>>> interested.
>>>>>>>>>>>>> kafka
>>>>>>>>>>>>>>>>>>>>> streams at
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              point supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> all sorts of joins as
>>>>>>> long as
>>>>>>>>>> both
>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> Adam is currently
>>>>>>>>> implementing a
>>>>>>>>>>>> join
>>>>>>>>>>>>>>>> where a
>>>>>>>>>>>>>>>>>>>>> KTable
>>>>>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              KTable can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> a one to many relation
>>>>>>> ship
>>>>>>>>>> (1:n).
>>>>>>>>>>>> We
>>>>>>>>>>>>>>>> exploit
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> datastore that keeps data
>>>>>>>>> sorted
>>>>>>>>>> (At
>>>>>>>>>>>>> least
>>>>>>>>>>>>>>>>>>>>> exposes an
>>>>>>>>>>>>>>>>>>>>>>>>>> API to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> stored data in a sorted
>>>>>>>>>> fashion).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> I think the technical
>>>>>>> caveats
>>>>>>>>>> are
>>>>>>>>>>>> well
>>>>>>>>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > basically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> down to philosophy and API
>>>>>>>>>> Design
>>>>>>>>>>> (
>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>> sees
>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>> newest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              message).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> I have a lengthy track
>>>>>>>>> record of
>>>>>>>>>>>>> loosing
>>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>>>> kinda
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              arguments within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> streams community and I
>>>>>>> have
>>>>>>>>> no
>>>>>>>>>>> clue
>>>>>>>>>>>>>> why.
>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> literally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              can't wait for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> to churn through this
>>>>>>> thread
>>>>>>>>> and
>>>>>>>>>>>> give
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> opinion on
>>>>>>>>>>>>>>>>>>>>>>>>>> how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> the return type of the
>>>>>>>>>>> oneToManyJoin
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>>>> power we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              want to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> the user vs "simplicity"
>>>>>>>>> (where
>>>>>>>>>>>>>> simplicity
>>>>>>>>>>>>>>>>>>> isn't
>>>>>>>>>>>>>>>>>>>>>>>>>> really that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              as users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> need to understand it I
>>>>>>>>> argue)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> waiting for you to join
>>>>>>> in on
>>>>>>>>>> the
>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> Best Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >> On 07.09.2018 15:49, James
>>>>>>>>> Kwan
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>> I am new to this group
>>>>>>> and I
>>>>>>>>>>> found
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> subject
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              interesting.  Sounds
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>> you guys want to
>>>>>>> implement a
>>>>>>>>>> join
>>>>>>>>>>>>>> table of
>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>>>> streams? Is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > somewhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>> I can see the original
>>>>>>>>>>> requirement
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> proposal?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>> On Sep 7, 2018, at 8:13
>>>>>>> AM,
>>>>>>>>> Jan
>>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <Jan.Filipiak@trivago.com
>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> On 05.09.2018 22:17,
>>>>>>> Adam
>>>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> I'm currently testing
>>>>>>>>> using a
>>>>>>>>>>>>>> Windowed
>>>>>>>>>>>>>>>>>>> Store
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> store the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              highwater
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> mark.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> By all indications this
>>>>>>>>>> should
>>>>>>>>>>>> work
>>>>>>>>>>>>>>>> fine,
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> caveat
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              being that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> only resolve
>>>>>>> out-of-order
>>>>>>>>>>> arrival
>>>>>>>>>>>>>> for up
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> size of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the window
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > (ie:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> 24h, 72h, etc). This
>>>>>>> would
>>>>>>>>>>> remove
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> possibility
>>>>>>>>>>>>>>>>>>>>>>>>>> of it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > unbounded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> size.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> With regards to Jan's
>>>>>>>>>>>> suggestion, I
>>>>>>>>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> remain in disagreement.
>>>>>>>>>> While I
>>>>>>>>>>>> do
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> disagree
>>>>>>>>>>>>>>>>>>>>>>>>>> with your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              statement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> there likely to be
>>>>>>>>> additional
>>>>>>>>>>>> joins
>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              workflow, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> see how you can
>>>>>>>>> conclusively
>>>>>>>>>>> deal
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> foreign-key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> changes and subsequent
>>>>>>>>>> joins. I
>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              think you have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> proposed (without a
>>>>>>>>>> high-water,
>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>> groupBy and
>>>>>>>>>>>>>>>>>>>>>>>>>> reduce)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              and found
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> that if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> the foreign key changes
>>>>>>>>> too
>>>>>>>>>>>>> quickly,
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> load
>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              stream thread
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> too
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> high, the joined
>>>>>>> messages
>>>>>>>>>> will
>>>>>>>>>>>>> arrive
>>>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>> and be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              incorrectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> propagated, such that
>>>>>>> an
>>>>>>>>>>>>> intermediate
>>>>>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> event.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> Can you shed some light
>>>>>>> on
>>>>>>>>>> your
>>>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              There must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> some sort of flaw in it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> I have a suspicion
>>>>>>> where it
>>>>>>>>>> is,
>>>>>>>>>>> I
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              confirm. The idea
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> is bullet proof and it
>>>>>>>>> must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> an implementation mess
>>>>>>> up.
>>>>>>>>> I
>>>>>>>>>>> would
>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> clarify
>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we draw a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> conclusion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>    Repartitioning the
>>>>>>>>>> scattered
>>>>>>>>>>>>> events
>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> partitions is the only
>>>>>>> way I
>>>>>>>>>> know
>>>>>>>>>>>> how
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> conclusively
>>>>>>>>>>>>>>>>>>>>>>>>>> deal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> out-of-order events in
>>>>>>> a
>>>>>>>>>> given
>>>>>>>>>>>> time
>>>>>>>>>>>>>>>> frame,
>>>>>>>>>>>>>>>>>>>>> and to
>>>>>>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> eventually consistent
>>>>>>> with
>>>>>>>>>> the
>>>>>>>>>>>>> input
>>>>>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> If you have some code
>>>>>>> to
>>>>>>>>>> share
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> illustrates
>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              approach, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> very grateful as it
>>>>>>> would
>>>>>>>>>>> remove
>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>> misunderstandings
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that I may
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > have.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> ah okay you were looking
>>>>>>>>> for
>>>>>>>>>> my
>>>>>>>>>>>>> code.
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              something easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> readable here as its
>>>>>>>>> bloated
>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> OO-patterns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> its anyhow trivial:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> @Override
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>      public T apply(K
>>>>>>>>> aggKey,
>>>>>>>>>> V
>>>>>>>>>>>>>> value, T
>>>>>>>>>>>>>>>>>>>>>>> aggregate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>      {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          Map<U, V>
>>>>>>>>>>>>> currentStateAsMap =
>>>>>>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              imaginary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          U toModifyKey =
>>>>>>>>>>>>>>>>>>> mapper.apply(value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              << this is
>>>>>>> the
>>>>>>>>>>> place
>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              gonna have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> and why you probably
>>>>>>>>> couldn't
>>>>>>>>>> do
>>>>>>>>>>>> it.
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>> to find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              a solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> I didn't realize that
>>>>>>> yet.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              << we
>>>>>>>>> propagate
>>>>>>>>>> the
>>>>>>>>>>>>>> field in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> joiner, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that we can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > pick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> it up in an aggregate.
>>>>>>>>>> Probably
>>>>>>>>>>>> you
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> thought
>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              this in your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> approach right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              << I am
>>>>>>> very
>>>>>>>>> open
>>>>>>>>>>> to
>>>>>>>>>>>>>> find a
>>>>>>>>>>>>>>>>>>>>> generic
>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              here. In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> honest opinion this is
>>>>>>>>> broken
>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              looses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > the keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> and only maintains the
>>>>>>>>>> aggregate
>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              << I
>>>>>>>>> abstracted
>>>>>>>>>> it
>>>>>>>>>>>> away
>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>> then way
>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> of oneToMany join. That
>>>>>>> is
>>>>>>>>>> why I
>>>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>>>> realize
>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              significance here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              <<
>>>>>>> Opinions?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          for (V m :
>>>>>>>>> current)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>> currentStateAsMap.put(mapper.apply(m),
>>>>>>>>>>>>>>>> m);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          if (isAdder)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>> currentStateAsMap.put(toModifyKey,
>>>>>>>>>>>>>>>> value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          else
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>                  return
>>>>>>>>> null;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>          retrun
>>>>>>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>      }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> On Wed, Sep 5, 2018 at
>>>>>>>>> 3:35
>>>>>>>>>> PM,
>>>>>>>>>>>> Jan
>>>>>>>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Jan.Filipiak@trivago.com
>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>> Thanks Adam for
>>>>>>> bringing
>>>>>>>>>>> Matthias
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> speed!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> about the
>>>>>>> differences. I
>>>>>>>>>> think
>>>>>>>>>>>>>>>> re-keying
>>>>>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              optional at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> best.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> I would say we return
>>>>>>> a
>>>>>>>>>>>>>> KScatteredTable
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> reshuffle()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>> KTable<originalKey,Joined>
>>>>>>>>>> to
>>>>>>>>>>>> make
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              repartitioning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> I am also in a big
>>>>>>>>> favour of
>>>>>>>>>>>> doing
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              processing using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> by instead high water
>>>>>>>>> mark
>>>>>>>>>>>>> tracking.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> Just because unbounded
>>>>>>>>>> growth
>>>>>>>>>>> is
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>> scary
>>>>>>>>>>>>>>>>>>>>> + It
>>>>>>>>>>>>>>>>>>>>>>>>>> saves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> stuff.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> I think the
>>>>>>> abstraction
>>>>>>>>> of
>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>>>>> repartitioning
>>>>>>>>>>>>>>>>>>>>>>>>>> back is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              just not so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> strong. Like the work
>>>>>>> has
>>>>>>>>>> been
>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              back and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> grouping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> by something else
>>>>>>>>> afterwards
>>>>>>>>>>> is
>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>> common.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> On 05.09.2018 13:49,
>>>>>>> Adam
>>>>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>> Hi Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Thank you for your
>>>>>>>>>> feedback,
>>>>>>>>>>> I
>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> While name spacing
>>>>>>>>> would be
>>>>>>>>>>>>>> possible,
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> user headers what
>>>>>>>>> implies
>>>>>>>>>> a
>>>>>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> namespace for now to
>>>>>>>>> avoid
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> the future, we can
>>>>>>>>> still
>>>>>>>>>> add
>>>>>>>>>>>>> name
>>>>>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Agreed. I will go
>>>>>>> with
>>>>>>>>>>> using a
>>>>>>>>>>>>>>>> reserved
>>>>>>>>>>>>>>>>>>>>> string
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              document it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> My main concern about
>>>>>>>>> the
>>>>>>>>>>>> design
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> type of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> understood the
>>>>>>> proposal
>>>>>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> In your example, you
>>>>>>>>> have
>>>>>>>>>>>> table1
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>>>>>> swapped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              Here is how it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> works
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> currently:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> 1) table1 has the
>>>>>>>>> records
>>>>>>>>>>> that
>>>>>>>>>>>>>> contain
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> foreign key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              within their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> value.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> table1 input stream:
>>>>>>>>>>>>>> <a,(fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> table2 input stream:
>>>>>>>>> <A,X>,
>>>>>>>>>>>> <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> 2) A Value mapper is
>>>>>>>>>> required
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> extract
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> table1 foreign key
>>>>>>>>> mapper:
>>>>>>>>>> (
>>>>>>>>>>>>> value
>>>>>>>>>>>>>> =>
>>>>>>>>>>>>>>>>>>>>> value.fk
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <http://value.fk> )
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> The mapper is
>>>>>>> applied to
>>>>>>>>>> each
>>>>>>>>>>>>>> element
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              new combined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> key is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> made:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> table1 mapped: <A-a,
>>>>>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <B-c,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> 3) The rekeyed events
>>>>>>>>> are
>>>>>>>>>>>>>>>> copartitioned
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> table2:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> a) Stream Thread with
>>>>>>>>>>> Partition
>>>>>>>>>>>>> 0:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> RepartitionedTable1:
>>>>>>>>> <A-a,
>>>>>>>>>>>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              (fk=A,bar=2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> b) Stream Thread with
>>>>>>>>>>> Partition
>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> RepartitionedTable1:
>>>>>>>>> <B-c,
>>>>>>>>>>>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> 4) From here, they
>>>>>>> can
>>>>>>>>> be
>>>>>>>>>>>> joined
>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              applying the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> joiner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> function.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> At this point, Jan's
>>>>>>>>> design
>>>>>>>>>>> and
>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>> deviate. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              design goes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> repartition the data
>>>>>>>>>>> post-join
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> resolve
>>>>>>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              arrival of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> records,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> finally returning the
>>>>>>>>> data
>>>>>>>>>>>> keyed
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> original key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              I do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> expose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> CombinedKey or any of
>>>>>>>>> the
>>>>>>>>>>>>> internals
>>>>>>>>>>>>>>>>>>>>> outside of
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              joinOnForeignKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> function. This does
>>>>>>> make
>>>>>>>>>> for
>>>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>>>> footprint,
>>>>>>>>>>>>>>>>>>>>>>>>>> but it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              removes all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> agency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> for resolving
>>>>>>>>> out-of-order
>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              CombinedKeys from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> user. I believe that
>>>>>>>>> this
>>>>>>>>>>> makes
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> function
>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to use.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Let me know if this
>>>>>>>>> helps
>>>>>>>>>>>> resolve
>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>> questions,
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              please feel
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> add anything else on
>>>>>>>>> your
>>>>>>>>>>> mind.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> On Tue, Sep 4, 2018
>>>>>>> at
>>>>>>>>> 8:36
>>>>>>>>>>> PM,
>>>>>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>> matthias@confluent.io
>>>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> I am just catching
>>>>>>> up
>>>>>>>>> on
>>>>>>>>>>> this
>>>>>>>>>>>>>>>> thread. I
>>>>>>>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              everything so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> far,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> but want to share
>>>>>>>>> couple
>>>>>>>>>> of
>>>>>>>>>>>>>> initial
>>>>>>>>>>>>>>>>>>>>> thoughts:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Headers: I think
>>>>>>> there
>>>>>>>>> is
>>>>>>>>>> a
>>>>>>>>>>>>>>>> fundamental
>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              between header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> usage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> in this KIP and
>>>>>>> KP-258.
>>>>>>>>>> For
>>>>>>>>>>>> 258,
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> are owned by Kafka
>>>>>>>>> Streams
>>>>>>>>>>> and
>>>>>>>>>>>>>> nobody
>>>>>>>>>>>>>>>>>>>>> else is
>>>>>>>>>>>>>>>>>>>>>>>>>> supposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> them. In fact, no
>>>>>>> user
>>>>>>>>>>> header
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> thus, there are not
>>>>>>>>>>> conflicts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Nevertheless, I
>>>>>>> don't
>>>>>>>>> see
>>>>>>>>>> a
>>>>>>>>>>>> big
>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              headers within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> As long as we
>>>>>>> document
>>>>>>>>> it,
>>>>>>>>>>> we
>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>> "reserved"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              header keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> users are not
>>>>>>> allowed
>>>>>>>>> to
>>>>>>>>>> use
>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>> data with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> IMHO, this should be
>>>>>>>>> ok.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> I think there is a
>>>>>>> safe
>>>>>>>>>> way
>>>>>>>>>>> to
>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>> conflicts,
>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > headers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> only needed in
>>>>>>>>> internal
>>>>>>>>>>>> topics
>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>>>> think):
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> For internal and
>>>>>>>>>> changelog
>>>>>>>>>>>>>> topics,
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>> namespace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              all headers:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> * user-defined
>>>>>>> headers
>>>>>>>>>> are
>>>>>>>>>>>>>>>> namespaced
>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>> "external."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> * internal headers
>>>>>>> are
>>>>>>>>>>>>>> namespaced as
>>>>>>>>>>>>>>>>>>>>>>>>>> "internal." +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> While name spacing
>>>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>>>>> possible,
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> user headers what
>>>>>>>>> implies
>>>>>>>>>> a
>>>>>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> namespace for now to
>>>>>>>>> avoid
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> the future, we can
>>>>>>>>> still
>>>>>>>>>> add
>>>>>>>>>>>>> name
>>>>>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> My main concern
>>>>>>> about
>>>>>>>>> the
>>>>>>>>>>>> design
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> If I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> understood the
>>>>>>> proposal
>>>>>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> KTable<K1,V1>
>>>>>>> table1 =
>>>>>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> KTable<K2,V2>
>>>>>>> table2 =
>>>>>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> KTable<K1,V3>
>>>>>>>>> joinedTable
>>>>>>>>>> =
>>>>>>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> implies that the
>>>>>>>>>>> `joinedTable`
>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              left input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> IMHO, this does not
>>>>>>>>> work
>>>>>>>>>>>> because
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>>>>>> contains
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              multiple rows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> join with a record
>>>>>>> in
>>>>>>>>>> table1
>>>>>>>>>>>>>> (what is
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>>>>>> purpose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> join), the result
>>>>>>> table
>>>>>>>>>>> would
>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>> contain a
>>>>>>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              join result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> multiple.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Example:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> table1 input stream:
>>>>>>>>> <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> table2 input stream:
>>>>>>>>>>>> <a,(A,1)>,
>>>>>>>>>>>>>>>>>>> <b,(A,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> We use table2 value
>>>>>>> a
>>>>>>>>>>> foreign
>>>>>>>>>>>>> key
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> table1
>>>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              "A" joins).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> result key is the
>>>>>>> same
>>>>>>>>> key
>>>>>>>>>>> as
>>>>>>>>>>>>> key
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              implies that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> result can either be
>>>>>>>>> <A,
>>>>>>>>>>>>>> join(X,1)>
>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> <A,
>>>>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > both.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Because the share
>>>>>>> the
>>>>>>>>> same
>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>> whatever
>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we emit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> overwrite the
>>>>>>> previous
>>>>>>>>>>> result.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> This is the reason
>>>>>>> why
>>>>>>>>> Jan
>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > combination
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> both primary keys of
>>>>>>>>> the
>>>>>>>>>>> input
>>>>>>>>>>>>>> tables
>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              output table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> makes the keys of
>>>>>>> the
>>>>>>>>>> output
>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>> unique
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              store both in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> output table:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Result would be
>>>>>>> <A-a,
>>>>>>>>>>>>> join(X,1)>,
>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> On 9/4/18 1:36 PM,
>>>>>>> Jan
>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>> Just on remark here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> The high-watermark
>>>>>>>>> could
>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> disregarded.
>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> depends on the
>>>>>>> size of
>>>>>>>>>> the
>>>>>>>>>>>>>>>> aggregated
>>>>>>>>>>>>>>>>>>>>> map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> Only 1 element long
>>>>>>>>> maps
>>>>>>>>>>>> would
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> unpacked
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              forwarded. 0
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > element
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> maps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> would be published
>>>>>>> as
>>>>>>>>>>> delete.
>>>>>>>>>>>>> Any
>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>> count
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> of map entries is
>>>>>>> in
>>>>>>>>>>> "waiting
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> correct
>>>>>>>>>>>>>>>>>>>>>>>>>> deletes to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > arrive"-state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> On 04.09.2018
>>>>>>> 21:29,
>>>>>>>>> Adam
>>>>>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> It does look like I
>>>>>>>>> could
>>>>>>>>>>>>> replace
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              repartition store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> highwater store
>>>>>>> with
>>>>>>>>> a
>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> reduce.
>>>>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              it looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> still need to
>>>>>>> store
>>>>>>>>> the
>>>>>>>>>>>>>> highwater
>>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>>>>>> within
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              materialized
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> compare the
>>>>>>> arrival of
>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>>>>> (assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> THIS is
>>>>>>> correct...).
>>>>>>>>> This
>>>>>>>>>>> in
>>>>>>>>>>>>>> effect
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              design I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>> just with the two
>>>>>>>>> tables
>>>>>>>>>>>> merged
>>>>>>>>>>>>>>>>>>> together.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> -- Guozhang
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Jan and Adam,

Good to hear from you both! Happy new year!

Hey Jan, I'm sorry you feel that we aren't listening to you. I was hoping
to convey that we did understand your proposal, but simply differed in
opinion at the end of the day,

I agree with your assessment that your proposal is "eventually correct",
which is the same guarantee that all the other proposals have offered.
It also seems like your proposal does create an opportunity for developers
to implement optimizations, particularly when chaining together multiple
joins.

API:

However, this feature does come at the expense of adding a whole new
KScatteredTable interface and the corresponding mental overhead of picturing
the data flow to make sense of it and understand how to use it in both
single
and multiple join scenarios. I suppose there's an argument that developers
should *always* bear in mind the underlying implementation details of the
systems they use. Practically speaking, though, it seems like the best
systems
are ones that don't require detailed internal knowledge to be used
productively.

I'm not sure how to express that I'm strongly sympathetic to the point of
view
that we shouldn't design to prohibit optimization. And that the promise of
unspecified future possible internal optimizations may never actually bear
fruit.

Efficiency:

In this case, I haven't seen or been able to build for myself a strong case
that
the scattered table API would actually be more efficient, even in the
presence
of multiple chained joins. It's true that you get to amortize the cost of
the post-join
group/gather operation over the number of chained joins. But on the other
hand,
it requires sending the higher-cardinality data over the wire, instead of
the
lower-cardinality data. And it requires maintaining a linear amount of
candidate join
result data (in the "map" in your example) to perform the final resolution.

I decided not to bring this up in the KIP discussion before, since it
sounded like
everyone liked the latest proposal, but there is an additional optimization
available
to the current proposal:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-Workflow

In step 2, we are only sending the primary and foreign key data over the
wire,
so if we really want to pare down the amount of traffic and computation, we
can
check first whether the FK has actually changed. If we also materialize
step 6, the
right kTable values, then we can avoid any network or right-side range
scans and simply
recompute the join result directly in the left-side Joiner (8) when there's
any change
on the left side data that doesn't affect the PK->FK mapping.

As you pointed out, we have to make some kind of decision in the absence of
any
good data regarding the actual data set or workload.

Here are the assumptions I'm currently operating with:
* the left side is higher cardinality than the right side
* the higher cardinality data would change more frequently
* (step 2) sending only PK+FK is smaller than sending the whole left-side
record
* broadcasting the right-side data only on right-side changes and FK
changes is
   less than broadcasting the join result on all changes
* comparing the records in step 6 for FK consistency with the left-side
table is
   cheaper than maintaining a map of all recent join results to resolve
disordering

These are all assumptions for sure, but I think they are reasonable ones.
Together, they mean that when you're joining just two tables, the
single-operator join is also
more efficient. I've been trying to do some math to determine if this holds
up for three
tables as well, but it's murky with all the unbound terms. Tentatively, I
actually do think
that the single-operator join is also likely to be more efficient for three
tables. Happy to provide
more detail if you don't buy this.

However, you seem to have a strong intuition that the scatter/gather
approach is better.
Is this informed by your actual applications at work? Perhaps you can
provide an example
data set and sequence of operations so we can all do the math and agree
with you.
It seems like we should have a convincing efficiency argument before
choosing a more
complicated API over a simpler one.

Last thought:
> Regarding what will be observed. I consider it a plus that all events
> that are in the inputs have an respective output. Whereas your solution
> might "swallow" events.

I didn't follow this. Following Adam's example, we have two join results: a
"dead" one and
a "live" one. If we get the dead one first, both solutions emit it,
followed by the live result.
If we get the dead result second, both solutions should suppress it. In your
proposal, both the dead and live result would be stored in that map, but
the groupBy operator
must not emit the dead result after the live one in any case. I guess you
were referring only to the
fact that the scattered table emits does not swallow any events? This seems
partially unrelated, since
the join is still incomplete at that point.

Thanks for your time to help us get this right!
-John

On Wed, Jan 2, 2019 at 2:36 PM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Jan
>
> Ahh, I got it! It is deterministic once you apply the groupBy function you
> mentioned a few months ago to the output, but not before you apply it...
> correct? I was not thinking about the groupBy function.
>
> Here's how I understand how it could work from an API perspective: I am
> going to use the terminology "KScatteredTable" to represent the
> intermediate table that is not yet resolved - basically the join was
> performed but no race condition handling is done.
>
> If I wanted to join three KTables together on foreign keys, one of the ways
> I could do it is:
>
> KScatteredTable scatteredOne =  ktableOne.oneToManyJoin(kTableTwo,
> joinerFuncTwo, foreignKeyExtractorTwo);
> KScatteredTable scatteredTwo = scatteredOne.oneToManyJoin(kTableThree,
> joinerFuncThree, foreignKeyExtractorThree)
>
> //Now I groupBy the key that I want to obtain, and I can resolve the out of
> order dependencies here.
> scatteredTwo.groupBy( keyValueMapper )   ( shown here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-SolutionB-User-ManagedGroupBy(Jan's)
> )
>
> Is this in line with what you're doing? Can this be done without exposing
> the CombinedKey? As you mentioned before "A Table
> KTable<CombinedKey<A,B>,JoinedResult> is not a good return type. It breaks
> the KTable invariant that a table is currently partitioned by its key".
> With that being said, are the only two operations that a KScatteredTable
> would need to support be oneToManyJoin and groupBy?
>
> Thanks for your thoughts
>
> Adam
>
>
> On Wed, Jan 2, 2019 at 3:07 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > Hi Adam,
> >
> > I am kinda surprised! Yes my solution of course is correct. Don't really
> > know what to show in an example as I am convinced you grabbed the
> > concept of how mine works,
> >
> > If there is a race condition there is a race condition. It doesn't
> > matter if there is 10 minutes or milliseconds between events. Either
> > they are properly guarded or not. My solution has no such race
> > condition. It is 'eventual consistent'. You gonna see all sort of stuff
> > coming up during a reprocess.
> >
> > The user can still fk it up later though. But that is usual business.
> >
> > In reality I try to supress updates from left sides as long as possible
> > because right side updates are more expensive if left is already
> > fullish. So that limits the space a little but there are no grantees.
> > The result however, after lag is zero is the same every time.
> >
> > The trade-offs can be shifted as you like. My solution gives full power
> > to the user and only does a minimum in the framework. You push
> > everything into streams.
> >
> > If you ask me, not a good choice. Will anyone listen. No.
> > I do actually think its to late to do my way. It's not like if you
> > haven't been gone through the effort and building it.
> >
> > Just wanted to give you guys another chance, to think it through  ;)
> >
> > Regarding what will be observed. I consider it a plus that all events
> > that are in the inputs have an respective output. Whereas your solution
> > might "swallow" events.
> >
> > Best Jan
> >
> >
> > On 02.01.2019 15:30, Adam Bellemare wrote:
> > > Jan
> > >
> > > I have been thinking a lot about the history of the discussion and your
> > > original proposal, and why you believe it is a better solution. The
> > biggest
> > > problem with your original proposed design is that it seems to me to be
> > > non-deterministic. It is subject to race conditions that are dependent
> > > entirely on the data, and without resolution of these races you can end
> > up
> > > with different results each time. If I am mistaken and this is indeed
> > > deterministic, then please let me know and provide an explanation,
> > ideally
> > > with an example.
> > >
> > > The way I see it is that you will get very different answers to your
> > > non-race-condition-resolved join topology, especially if you are
> nesting
> > it
> > > with additional joins as you have indicated you are doing. Consider
> > > rebuilding an application state from the beginning of two topics. If
> the
> > > left/this side has multiple foreign-key changes in a row, spaced out
> > every
> > > ten minutes, you may see something like this:
> > >
> > > (foo, foreignKey=red) t=0
> > > (foo, foreignKey=blue) t=0+10m
> > > (foo, foreignKey=green) t=0+20m
> > > (foo, foreignKey=purple) t=0+30m
> > > (foo, foreignKey=blue) t=0+40m
> > > (foo, foreignKey=white) t=0+50m
> > >
> > > During realtime processing, all of the updates may have correctly
> > > propagated because it took less than 10 minutes to resolve each join.
> > Upon
> > > rebuilding from the start, however, all of these events would be
> > processed
> > > in quick succession. The presence or absence of data will affect the
> > > results of your join, and the results can vary with each run depending
> on
> > > the data. Because of this, I cannot support any kind of solution that
> > would
> > > allow the exposure of an unresolved intermediate state. I can
> understand
> > if
> > > you don't support this, but this is why, as you said, you have the
> > freedom
> > > to use the Processor API.
> > >
> > >
> > > With that being said, either the solution that I originally proposed
> > > (join's ocurring on the foreign node) or John + Guozhang's solution
> > > (registering with the foreign node for notifications) is fine with me -
> > > both have the same API and we can evaluate it further during
> > implementation.
> > >
> > >
> > > Thanks
> > >
> > > Adam
> > >
> > > On Thu, Dec 27, 2018 at 2:38 PM Jan Filipiak <Jan.Filipiak@trivago.com
> >
> > > wrote:
> > >
> > >> Hi,
> > >>
> > >> just want to let you guys know that this thing is spiralling out of
> > >> control if you ask me.
> > >>
> > >> First you take away the possibility for the user to optimize. Now you
> > >> pile up complexity to perform some afterwards optimisation, that from
> my
> > >> POV completely misses the point. As if the actual call to the joiner
> > >> really gonna be an expensive part. It wont. Truth is, you don't have a
> > >> clue which side is gonna be smaller. might be the key you shuffle
> around
> > >> is >>> than the value on the other side already.
> > >>
> > >> You know my opinion on this. For me its dead, I just leave you the
> > >> message here as an opportunity to reconsider the choices that were
> made.
> > >>
> > >> Whish y'll a happy new year :)
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On 27.12.2018 17:22, Adam Bellemare wrote:
> > >>> Hi All
> > >>>
> > >>> Sorry for the delay - holidays and all. I have since updated the KIP
> > with
> > >>> John's original suggestion and have pruned a number of the no longer
> > >>> relevant diagrams. Any more comments would be welcomed, otherwise I
> > will
> > >>> look to kick off the vote again shortly.
> > >>>
> > >>> Thanks
> > >>> Adam
> > >>>
> > >>> On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <
> > adam.bellemare@gmail.com
> > >>>
> > >>> wrote:
> > >>>
> > >>>> Hi John and Guozhang
> > >>>>
> > >>>> Ah yes, I lost that in the mix! Thanks for the convergent solutions
> -
> > I
> > >> do
> > >>>> think that the attachment that John included makes for a better
> > design.
> > >> It
> > >>>> should also help with overall performance as very high-cardinality
> > >> foreign
> > >>>> keyed data (say millions of events with the same entity) will be
> able
> > to
> > >>>> leverage the multiple nodes for join functionality instead of having
> > it
> > >> all
> > >>>> performed in one node. There is still a bottleneck in the right
> table
> > >>>> having to propagate all those events, but with slimmer structures,
> > less
> > >> IO
> > >>>> and no need to perform the join I think the throughput will be much
> > >> higher
> > >>>> in those scenarios.
> > >>>>
> > >>>> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
> > >>>> version of John's diagram and ensure that the example flow matches
> > >>>> correctly. Then I can go back to working on the PR to match the
> > diagram.
> > >>>>
> > >>>> Thanks both of you for all the help - very much appreciated.
> > >>>>
> > >>>> Adam
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >>>>
> > >>>>> Hi John,
> > >>>>>
> > >>>>> Just made a pass on your diagram (nice hand-drawing btw!), and
> > >> obviously
> > >>>>> we
> > >>>>> are thinking about the same thing :) A neat difference that I like,
> > is
> > >>>>> that
> > >>>>> in the pre-join repartition topic we can still send message in the
> > >> format
> > >>>>> of `K=k, V=(i=2)` while using "i" as the partition key in
> > >>>>> StreamsPartition,
> > >>>>> this way we do not need to even augment the key for the repartition
> > >> topic,
> > >>>>> but just do a projection on the foreign key part but trim all other
> > >>>>> fields:
> > >>>>> as long as we still materialize the store as `A-2` co-located with
> > the
> > >>>>> right KTable, that is fine.
> > >>>>>
> > >>>>> As I mentioned in my previous email, I also think this has a few
> > >>>>> advantages
> > >>>>> on saving over-the-wire bytes as well as disk bytes.
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io>
> > >> wrote:
> > >>>>>
> > >>>>>> Hi Guozhang,
> > >>>>>>
> > >>>>>> Thanks for taking a look! I think Adam's already addressed your
> > >>>>> questions
> > >>>>>> as well as I could have.
> > >>>>>>
> > >>>>>> Hi Adam,
> > >>>>>>
> > >>>>>> Thanks for updating the KIP. It looks great, especially how all
> the
> > >>>>>> need-to-know information is right at the top, followed by the
> > details.
> > >>>>>>
> > >>>>>> Also, thanks for that high-level diagram. Actually, now that I'm
> > >> looking
> > >>>>>> at it, I think part of my proposal got lost in translation,
> > although I
> > >>>>> do
> > >>>>>> think that what you have there is also correct.
> > >>>>>>
> > >>>>>> I sketched up a crude diagram based on yours and attached it to
> the
> > >> KIP
> > >>>>>> (I'm not sure if attached or inline images work on the mailing
> > list):
> > >>>>>>
> > >>>>>
> > >>
> >
> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
> > >>>>>> . It's also attached to this email for convenience.
> > >>>>>>
> > >>>>>> Hopefully, you can see how it's intended to line up, and which
> parts
> > >> are
> > >>>>>> modified.
> > >>>>>> At a high level, instead of performing the join on the right-hand
> > >> side,
> > >>>>>> we're essentially just registering interest, like "LHS key A
> wishes
> > to
> > >>>>>> receive updates for RHS key 2". Then, when there is a new
> "interest"
> > >> or
> > >>>>> any
> > >>>>>> updates to the RHS records, it "broadcasts" its state back to the
> > LHS
> > >>>>>> records who are interested in it.
> > >>>>>>
> > >>>>>> Thus, instead of sending the LHS values to the RHS joiner workers
> > and
> > >>>>> then
> > >>>>>> sending the join results back to the LHS worke be co-partitioned
> and
> > >>>>>> validated, we instead only send the LHS *keys* to the RHS workers
> > and
> > >>>>> then
> > >>>>>> only the RHS k/v back to be joined by the LHS worker.
> > >>>>>>
> > >>>>>> I've been considering both your diagram and mine, and I *think*
> what
> > >> I'm
> > >>>>>> proposing has a few advantages.
> > >>>>>>
> > >>>>>> Here are some points of interest as you look at the diagram:
> > >>>>>> * When we extract the foreign key and send it to the Pre-Join
> > >>>>> Repartition
> > >>>>>> Topic, we can send only the FK/PK pair. There's no need to worry
> > about
> > >>>>>> custom partitioner logic, since we can just use the foreign key
> > >> plainly
> > >>>>> as
> > >>>>>> the repartition record key. Also, we save on transmitting the LHS
> > >> value,
> > >>>>>> since we only send its key in this step.
> > >>>>>> * We also only need to store the RHSKey:LHSKey mapping in the
> > >>>>>> MaterializedSubscriptionStore, saving on disk. We can use the same
> > >> rocks
> > >>>>>> key format you proposed and the same algorithm involving range
> scans
> > >>>>> when
> > >>>>>> the RHS records get updated.
> > >>>>>> * Instead of joining on the right side, all we do is compose a
> > >>>>>> re-repartition record so we can broadcast the RHS k/v pair back to
> > the
> > >>>>>> original LHS partition. (this is what the "rekey" node is doing)
> > >>>>>> * Then, there is a special kind of Joiner that's co-resident in
> the
> > >> same
> > >>>>>> StreamTask as the LHS table, subscribed to the Post-Join
> Repartition
> > >>>>> Topic.
> > >>>>>> ** This Joiner is *not* triggered directly by any changes in the
> LHS
> > >>>>>> KTable. Instead, LHS events indirectly trigger the join via the
> > whole
> > >>>>>> lifecycle.
> > >>>>>> ** For each event arriving from the Post-Join Repartition Topic,
> the
> > >>>>>> Joiner looks up the corresponding record in the LHS KTable. It
> > >> validates
> > >>>>>> the FK as you noted, discarding any inconsistent events.
> Otherwise,
> > it
> > >>>>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the
> > >> join
> > >>>>>> result
> > >>>>>> ** Note that the Joiner itself is stateless, so materializing the
> > join
> > >>>>>> result is optional, just as with the 1:1 joins.
> > >>>>>>
> > >>>>>> So in summary:
> > >>>>>> * instead of transmitting the LHS keys and values to the right and
> > the
> > >>>>>> JoinResult back to the left, we only transmit the LHS keys to the
> > >> right
> > >>>>> and
> > >>>>>> the RHS values to the left. Assuming the average RHS value is on
> > >> smaller
> > >>>>>> than or equal to the average join result size, it's a clear win on
> > >>>>> broker
> > >>>>>> traffic. I think this is actually a reasonable assumption, which
> we
> > >> can
> > >>>>>> discuss more if you're suspicious.
> > >>>>>> * we only need one copy of the data (the left and right tables
> need
> > to
> > >>>>> be
> > >>>>>> materialized) and one extra copy of the PK:FK pairs in the
> > >> Materialized
> > >>>>>> Subscription Store. Materializing the join result is optional,
> just
> > as
> > >>>>> with
> > >>>>>> the existing 1:1 joins.
> > >>>>>> * we still need the fancy range-scan algorithm on the right to
> > locate
> > >>>>> all
> > >>>>>> interested LHS keys when a RHS value is updated, but we don't
> need a
> > >>>>> custom
> > >>>>>> partitioner for either repartition topic (this is of course a
> > >>>>> modification
> > >>>>>> we could make to your version as well)
> > >>>>>>
> > >>>>>> How does this sound to you? (And did I miss anything?)
> > >>>>>> -John
> > >>>>>>
> > >>>>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
> > >>>>> adam.bellemare@gmail.com>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Hi John & Guozhang
> > >>>>>>>
> > >>>>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up
> > the
> > >>>>> KIP,
> > >>>>>>> pruned much of what I wrote and put a simplified diagram near the
> > top
> > >>>>> to
> > >>>>>>> illustrate the workflow. I encapsulated Jan's content at the
> bottom
> > >> of
> > >>>>> the
> > >>>>>>> document. I believe it is simpler to read by far now.
> > >>>>>>>
> > >>>>>>> @Guozhang Wang <wa...@gmail.com>:
> > >>>>>>>> #1: rekey left table
> > >>>>>>>>     -> source from the left upstream, send to rekey-processor to
> > >>>>> generate
> > >>>>>>> combined key, and then sink to copartition topic.
> > >>>>>>> Correct.
> > >>>>>>>
> > >>>>>>>> #2: first-join with right table
> > >>>>>>>>     -> source from the right table upstream, materialize the
> right
> > >>>>> table.
> > >>>>>>>>     -> source from the co-partition topic, materialize the
> rekeyed
> > >> left
> > >>>>>>> table, join with the right table, rekey back, and then sink to
> the
> > >>>>>>> rekeyed-back topic.
> > >>>>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I
> need
> > >> the
> > >>>>>>> Foreign-Key value generated in #1 above to compare in the
> > resolution
> > >>>>>>> stage.
> > >>>>>>>
> > >>>>>>>> #3: second join
> > >>>>>>>>      -> source from the rekeyed-back topic, materialize the
> > rekeyed
> > >>>>> back
> > >>>>>>> table.
> > >>>>>>>>     -> source from the left upstream, materialize the left
> table,
> > >> join
> > >>>>>>> with
> > >>>>>>> the rekeyed back table.
> > >>>>>>> Almost - As each event comes in, we just run it through a
> stateful
> > >>>>>>> processor that checks the original ("This") KTable for the key.
> The
> > >>>>> value
> > >>>>>>> payload then has the foreignKeyExtractor applied again as in Part
> > #1
> > >>>>>>> above,
> > >>>>>>> and gets the current foreign key. Then we compare it to the
> joined
> > >>>>> event
> > >>>>>>> that we are currently resolving. If they have the same
> foreign-key,
> > >>>>>>> propagate the result out. If they don't, throw the event away.
> > >>>>>>>
> > >>>>>>> The end result is that we do need to materialize 2 additional
> > tables
> > >>>>>>> (left/this-combinedkey table, and the final Joined table) as I've
> > >>>>>>> illustrated in the updated KIP. I hope the diagram clears it up a
> > lot
> > >>>>>>> better. Please let me know.
> > >>>>>>>
> > >>>>>>> Thanks again
> > >>>>>>> Adam
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <
> wangguoz@gmail.com>
> > >>>>> wrote:
> > >>>>>>>
> > >>>>>>>> John,
> > >>>>>>>>
> > >>>>>>>> Thanks a lot for the suggestions on refactoring the wiki, I
> agree
> > >>>>> with
> > >>>>>>> you
> > >>>>>>>> that we should consider the KIP proposal to be easily understood
> > by
> > >>>>>>> anyone
> > >>>>>>>> in the future to read, and hence should provide a good summary
> on
> > >> the
> > >>>>>>>> user-facing interfaces, as well as rejected alternatives to
> > >> represent
> > >>>>>>>> briefly "how we came a long way to this conclusion, and what we
> > have
> > >>>>>>>> argued, disagreed, and agreed about, etc" so that readers do not
> > >>>>> need to
> > >>>>>>>> dig into the DISCUSS thread to get all the details. We can, of
> > >>>>> course,
> > >>>>>>> keep
> > >>>>>>>> the implementation details like "workflows" on the wiki page as
> a
> > >>>>>>> addendum
> > >>>>>>>> section since it also has correlations.
> > >>>>>>>>
> > >>>>>>>> Regarding your proposal on comment 6): that's a very interesting
> > >>>>> idea!
> > >>>>>>> Just
> > >>>>>>>> to clarify that I understands it fully correctly: the proposal's
> > >>>>>>> resulted
> > >>>>>>>> topology is still the same as the current proposal, where we
> will
> > >>>>> have 3
> > >>>>>>>> sub-topologies for this operator:
> > >>>>>>>>
> > >>>>>>>> #1: rekey left table
> > >>>>>>>>      -> source from the left upstream, send to rekey-processor
> to
> > >>>>> generate
> > >>>>>>>> combined key, and then sink to copartition topic.
> > >>>>>>>>
> > >>>>>>>> #2: first-join with right table
> > >>>>>>>>      -> source from the right table upstream, materialize the
> > right
> > >>>>> table.
> > >>>>>>>>      -> source from the co-partition topic, materialize the
> > rekeyed
> > >>>>> left
> > >>>>>>>> table, join with the right table, rekey back, and then sink to
> the
> > >>>>>>>> rekeyed-back topic.
> > >>>>>>>>
> > >>>>>>>> #3: second join
> > >>>>>>>>      -> source from the rekeyed-back topic, materialize the
> > rekeyed
> > >>>>> back
> > >>>>>>>> table.
> > >>>>>>>>      -> source from the left upstream, materialize the left
> table,
> > >> join
> > >>>>>>> with
> > >>>>>>>> the rekeyed back table.
> > >>>>>>>>
> > >>>>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology
> > since
> > >>>>>>> both of
> > >>>>>>>> them read from the left table source stream. In this workflow,
> we
> > >>>>> need
> > >>>>>>> to
> > >>>>>>>> materialize 4 tables (left table in #3, right table in #2,
> rekeyed
> > >>>>> left
> > >>>>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
> > >>>>>>>> (copartition topic, rekeyed-back topic).
> > >>>>>>>>
> > >>>>>>>> Compared with Adam's current proposal in the workflow overview,
> it
> > >>>>> has
> > >>>>>>> the
> > >>>>>>>> same num.materialize tables (left table, rekeyed left table,
> right
> > >>>>>>> table,
> > >>>>>>>> out-of-ordering resolver table), and same num.internal topics
> > (two).
> > >>>>> The
> > >>>>>>>> advantage is that on the copartition topic, we can save
> bandwidth
> > by
> > >>>>> not
> > >>>>>>>> sending value, and in #2 the rekeyed left table is smaller since
> > we
> > >>>>> do
> > >>>>>>> not
> > >>>>>>>> have any values to materialize. Is that right?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Guozhang
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <john@confluent.io
> >
> > >>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Hi Adam,
> > >>>>>>>>>
> > >>>>>>>>> Given that the committers are all pretty busy right now, I
> think
> > >>>>> that
> > >>>>>>> it
> > >>>>>>>>> would help if you were to refactor the KIP a little to reduce
> the
> > >>>>>>>> workload
> > >>>>>>>>> for reviewers.
> > >>>>>>>>>
> > >>>>>>>>> I'd recommend the following changes:
> > >>>>>>>>> * relocate all internal details to a section at the end called
> > >>>>>>> something
> > >>>>>>>>> like "Implementation Notes" or something like that.
> > >>>>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
> > >>>>> mention
> > >>>>>>>> only
> > >>>>>>>>> publicly-facing API changes.
> > >>>>>>>>> ** for example, the interface that you've already listed there,
> > as
> > >>>>>>> well
> > >>>>>>>> as
> > >>>>>>>>> a textual description of the guarantees we'll be providing
> (join
> > >>>>>>> result
> > >>>>>>>> is
> > >>>>>>>>> copartitioned with the LHS, and the join result is guaranteed
> > >>>>> correct)
> > >>>>>>>>>
> > >>>>>>>>> A good target would be that the whole main body of the KIP,
> > >>>>> including
> > >>>>>>>>> Status, Motivation, Proposal, Justification, and Rejected
> > >>>>> Alternatives
> > >>>>>>>> all
> > >>>>>>>>> fit "above the fold" (i.e., all fit on the screen at a
> > comfortable
> > >>>>>>> zoom
> > >>>>>>>>> level).
> > >>>>>>>>> I think the only real Rejected Alternative that bears mention
> at
> > >>>>> this
> > >>>>>>>> point
> > >>>>>>>>> is KScatteredTable, which you could just include the executive
> > >>>>>>> summary on
> > >>>>>>>>> (no implementation details), and link to extra details in the
> > >>>>>>>>> Implementation Notes section.
> > >>>>>>>>>
> > >>>>>>>>> Taking a look at the wiki page, ~90% of the text there is
> > internal
> > >>>>>>>> detail,
> > >>>>>>>>> which is useful for the dubious, but doesn't need to be
> ratified
> > >>>>> in a
> > >>>>>>>> vote
> > >>>>>>>>> (and would be subject to change without notice in the future
> > >>>>> anyway).
> > >>>>>>>>> There's also a lot of conflicting discussion, as you've very
> > >>>>>>> respectfully
> > >>>>>>>>> tried to preserve the original proposal from Jan while adding
> > your
> > >>>>>>> own.
> > >>>>>>>>> Isolating all this information in a dedicated section at the
> > bottom
> > >>>>>>> frees
> > >>>>>>>>> the voters up to focus on the public API part of the proposal,
> > >>>>> which
> > >>>>>>> is
> > >>>>>>>>> really all they need to consider.
> > >>>>>>>>>
> > >>>>>>>>> Plus, it'll be clear to future readers which parts of the
> > document
> > >>>>> are
> > >>>>>>>>> enduring, and which parts are a snapshot of our implementation
> > >>>>>>> thinking
> > >>>>>>>> at
> > >>>>>>>>> the time.
> > >>>>>>>>>
> > >>>>>>>>> I'm suggesting this because I suspect that the others haven't
> > made
> > >>>>>>> time
> > >>>>>>>> to
> > >>>>>>>>> review it partly because it seems daunting. If it seems like it
> > >>>>> would
> > >>>>>>> be
> > >>>>>>>> a
> > >>>>>>>>> huge time investment to review, people will just keep putting
> it
> > >>>>> off.
> > >>>>>>> But
> > >>>>>>>>> if the KIP is a single page, then they'll be more inclined to
> > give
> > >>>>> it
> > >>>>>>> a
> > >>>>>>>>> read.
> > >>>>>>>>>
> > >>>>>>>>> Honestly, I don't think the KIP itself is that controversial
> > (apart
> > >>>>>>> from
> > >>>>>>>>> the scattered table thing (sorry, Jan) ). Most of the
> discussion
> > >>>>> has
> > >>>>>>> been
> > >>>>>>>>> around the implementation, which we can continue more
> effectively
> > >>>>> in
> > >>>>>>> a PR
> > >>>>>>>>> once the KIP has passed.
> > >>>>>>>>>
> > >>>>>>>>> How does that sound?
> > >>>>>>>>> -John
> > >>>>>>>>>
> > >>>>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
> > >>>>>>> adam.bellemare@gmail.com
> > >>>>>>>>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> 1) I believe that the resolution mechanism John has proposed
> is
> > >>>>>>>>> sufficient
> > >>>>>>>>>> - it is clean and easy and doesn't require additional RocksDB
> > >>>>>>> stores,
> > >>>>>>>>> which
> > >>>>>>>>>> reduces the footprint greatly. I don't think we need to
> resolve
> > >>>>>>> based
> > >>>>>>>> on
> > >>>>>>>>>> timestamp or offset anymore, but if we decide to do to that
> > >>>>> would be
> > >>>>>>>>> within
> > >>>>>>>>>> the bounds of the existing API.
> > >>>>>>>>>>
> > >>>>>>>>>> 2) Is the current API sufficient, or does it need to be
> altered
> > >>>>> to
> > >>>>>>> go
> > >>>>>>>>> back
> > >>>>>>>>>> to vote?
> > >>>>>>>>>>
> > >>>>>>>>>> 3) KScatteredTable implementation can always be added in a
> > future
> > >>>>>>>>> revision.
> > >>>>>>>>>> This API does not rule it out. This implementation of this
> > >>>>> function
> > >>>>>>>> would
> > >>>>>>>>>> simply be replaced with `KScatteredTable.resolve()` while
> still
> > >>>>>>>>> maintaining
> > >>>>>>>>>> the existing API, thereby giving both features as Jan outlined
> > >>>>>>> earlier.
> > >>>>>>>>>> Would this work?
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Thanks Guozhang, John and Jan
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <
> > john@confluent.io
> > >>>>>>
> > >>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Hi, all,
> > >>>>>>>>>>>
> > >>>>>>>>>>>>> In fact, we
> > >>>>>>>>>>>>> can just keep a single final-result store with timestamps
> > >>>>> and
> > >>>>>>>> reject
> > >>>>>>>>>>> values
> > >>>>>>>>>>>>> that have a smaller timestamp, is that right?
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Which is the correct output should at least be decided on
> the
> > >>>>>>>> offset
> > >>>>>>>>> of
> > >>>>>>>>>>>> the original message.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks for this point, Jan.
> > >>>>>>>>>>>
> > >>>>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
> > >>>>> the
> > >>>>>>> k/v
> > >>>>>>>>>>> store,
> > >>>>>>>>>>> as well as providing a storage-format upgrade path.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I might have missed it, but I think we have yet to discuss
> > >>>>> whether
> > >>>>>>>> it's
> > >>>>>>>>>>> safe
> > >>>>>>>>>>> or desirable just to swap topic-ordering our for
> > >>>>>>> timestamp-ordering.
> > >>>>>>>>> This
> > >>>>>>>>>>> is
> > >>>>>>>>>>> a very deep topic, and I think it would only pollute the
> > >>>>> current
> > >>>>>>>>>>> discussion.
> > >>>>>>>>>>>
> > >>>>>>>>>>> What Adam has proposed is safe, given the *current* ordering
> > >>>>>>>> semantics
> > >>>>>>>>>>> of the system. If we can agree on his proposal, I think we
> can
> > >>>>>>> merge
> > >>>>>>>>> the
> > >>>>>>>>>>> feature well before the conversation about timestamp ordering
> > >>>>> even
> > >>>>>>>>> takes
> > >>>>>>>>>>> place, much less reaches a conclusion. In the mean time, it
> > >>>>> would
> > >>>>>>>> seem
> > >>>>>>>>> to
> > >>>>>>>>>>> be unfortunate to have one join operator with different
> > >>>>> ordering
> > >>>>>>>>>> semantics
> > >>>>>>>>>>> from every other KTable operator.
> > >>>>>>>>>>>
> > >>>>>>>>>>> If and when that timestamp discussion takes place, many
> (all?)
> > >>>>>>> KTable
> > >>>>>>>>>>> operations
> > >>>>>>>>>>> will need to be updated, rendering the many:one join a small
> > >>>>>>> marginal
> > >>>>>>>>>> cost.
> > >>>>>>>>>>>
> > >>>>>>>>>>> And, just to plug it again, I proposed an algorithm above
> that
> > >>>>> I
> > >>>>>>>>> believe
> > >>>>>>>>>>> provides
> > >>>>>>>>>>> correct ordering without any additional metadata, and
> > >>>>> regardless
> > >>>>>>> of
> > >>>>>>>> the
> > >>>>>>>>>>> ordering semantics. I didn't bring it up further, because I
> > >>>>> felt
> > >>>>>>> the
> > >>>>>>>>> KIP
> > >>>>>>>>>>> only needs
> > >>>>>>>>>>> to agree on the public API, and we can discuss the
> > >>>>> implementation
> > >>>>>>> at
> > >>>>>>>>>>> leisure in
> > >>>>>>>>>>> a PR...
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>> -John
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> > >>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
> > >>>>>>>>>>>>> Hello Adam / Jan / John,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Sorry for being late on this thread! I've finally got some
> > >>>>>>> time
> > >>>>>>>>> this
> > >>>>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
> > >>>>> I've
> > >>>>>>>> also
> > >>>>>>>>>>>> realized
> > >>>>>>>>>>>>> there are a bunch of other things I need to enqueue while
> > >>>>>>>> cleaning
> > >>>>>>>>>> them
> > >>>>>>>>>>>> up
> > >>>>>>>>>>>>> --- sth I need to improve on my side). So here are my
> > >>>>>>> thoughts:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regarding the APIs: I like the current written API in the
> > >>>>> KIP.
> > >>>>>>>> More
> > >>>>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
> > >>>>>>>>> functionalities
> > >>>>>>>>>> as
> > >>>>>>>>>>>>> well as 2) other join types than inner as separate KIPs
> > >>>>> since
> > >>>>>>> 1)
> > >>>>>>>>> may
> > >>>>>>>>>>>> worth
> > >>>>>>>>>>>>> a general API refactoring that can benefit not only
> > >>>>> foreignkey
> > >>>>>>>>> joins
> > >>>>>>>>>>> but
> > >>>>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > >>>>>>>>>>>> ),
> > >>>>>>>>>>>>> and I'm not sure if other join types would actually be
> > >>>>> needed
> > >>>>>>>>> (maybe
> > >>>>>>>>>>> left
> > >>>>>>>>>>>>> join still makes sense), so it's better to
> > >>>>>>>>>>> wait-for-people-to-ask-and-add
> > >>>>>>>>>>>>> than add-sth-that-no-one-uses.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
> > >>>>> a
> > >>>>>>>>>>>>> KScatteredTable for users to inject their own optimization:
> > >>>>>>> I'd
> > >>>>>>>>>> prefer
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>> do the current option as-is, and my main rationale is for
> > >>>>>>>>>> optimization
> > >>>>>>>>>>>>> rooms inside the Streams internals and the API
> > >>>>> succinctness.
> > >>>>>>> For
> > >>>>>>>>>>> advanced
> > >>>>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
> > >>>>> own
> > >>>>>>>>>>>> optimization,
> > >>>>>>>>>>>>> while it is too much of the work to use Processor API
> > >>>>>>> directly, I
> > >>>>>>>>>> think
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>> can still extend the current API to support it in the
> > >>>>> future
> > >>>>>>> if
> > >>>>>>>> it
> > >>>>>>>>>>>> becomes
> > >>>>>>>>>>>>> necessary.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> no internal optimization potential. it's a myth
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> ¯\_(ツ)_/¯
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> :-)
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
> > >>>>> as
> > >>>>>>> I
> > >>>>>>>>>>> mentioned
> > >>>>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
> > >>>>> key-value
> > >>>>>>>>> store)
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>> can actually make this step simpler than the current
> > >>>>>>> proposal. In
> > >>>>>>>>>> fact,
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>> can just keep a single final-result store with timestamps
> > >>>>> and
> > >>>>>>>>> reject
> > >>>>>>>>>>>> values
> > >>>>>>>>>>>>> that have a smaller timestamp, is that right?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Which is the correct output should at least be decided on
> the
> > >>>>>>>> offset
> > >>>>>>>>> of
> > >>>>>>>>>>>> the original message.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
> > >>>>>>> Adam
> > >>>>>>>> to
> > >>>>>>>>>>> make
> > >>>>>>>>>>>>> such HUGE progress on this KIP!
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> > >>>>>>>>>> Jan.Filipiak@trivago.com>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> If they don't find the time:
> > >>>>>>>>>>>>>> They usually take the opposite path from me :D
> > >>>>>>>>>>>>>> so the answer would be clear.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> hence my suggestion to vote.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>> Hi Guozhang and Matthias
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
> > >>>>> KIP
> > >>>>>>>> to a
> > >>>>>>>>>>> point
> > >>>>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
> > >>>>> a
> > >>>>>>>>>>> tie-breaker,
> > >>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>> you will). If you have anyone else you may think should
> > >>>>>>> look at
> > >>>>>>>>>> this,
> > >>>>>>>>>>>>>>> please tag them accordingly.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> The scenario is as such:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Current Option:
> > >>>>>>>>>>>>>>> API:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > >>>>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
> > >>>>>>>>> partition
> > >>>>>>>>>>> with
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> foreignKey (repartition 1)
> > >>>>>>>>>>>>>>> 2) Join the data
> > >>>>>>>>>>>>>>> 3) Shuffle the data back to the original node
> > >>>>> (repartition
> > >>>>>>> 2)
> > >>>>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
> > >>>>>>>>> foreign-key
> > >>>>>>>>>>>>>> changes.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Alternate Option:
> > >>>>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
> > >>>>>>>>>>>>>>> - It would be keyed on a wrapped key function:
> > >>>>>>> <CombinedKey<KO,
> > >>>>>>>>> K>,
> > >>>>>>>>>>> VR>
> > >>>>>>>>>>>>>> (KO
> > >>>>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
> > >>>>> Result)
> > >>>>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
> > >>>>>>>>> otherwise a
> > >>>>>>>>>>>> user
> > >>>>>>>>>>>>>>> would be able to perform additional functions directly
> > >>>>> from
> > >>>>>>> the
> > >>>>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
> > >>>>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
> > >>>>>>> tradeoffs.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
> > >>>>> possible,
> > >>>>>>>> but
> > >>>>>>>>>> will
> > >>>>>>>>>>>>>>> require for implementation details to be made in the API
> > >>>>> and
> > >>>>>>>> some
> > >>>>>>>>>>>>>> exposure
> > >>>>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I appreciate any insight into this.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > >>>>>>>>>>>> adam.bellemare@gmail.com>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hi John
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
> > >>>>>>> summary
> > >>>>>>>> is
> > >>>>>>>>>>>>>> accurate
> > >>>>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
> > >>>>> that
> > >>>>>>>>> there
> > >>>>>>>>>>> is a
> > >>>>>>>>>>>>>> risk
> > >>>>>>>>>>>>>>>> of inconsistent final states without performing the
> > >>>>>>>> resolution.
> > >>>>>>>>>> This
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>> major concern for me as most of the data I have dealt
> > >>>>> with
> > >>>>>>> is
> > >>>>>>>>>>> produced
> > >>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>> relational databases. We have seen a number of cases
> > >>>>> where
> > >>>>>>> a
> > >>>>>>>>> user
> > >>>>>>>>>> in
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
> > >>>>>>> they
> > >>>>>>>>> made
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>> mistake, and then updated the field again with a new
> > >>>>> key.
> > >>>>>>> The
> > >>>>>>>>>> events
> > >>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>> propagated out as they are produced, and as such we have
> > >>>>>>> had
> > >>>>>>>>>>>> real-world
> > >>>>>>>>>>>>>>>> cases where these inconsistencies were propagated
> > >>>>>>> downstream
> > >>>>>>>> as
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> final
> > >>>>>>>>>>>>>>>> values due to the race conditions in the fanout of the
> > >>>>>>> data.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> This solution that I propose values correctness of the
> > >>>>>>> final
> > >>>>>>>>>> result
> > >>>>>>>>>>>> over
> > >>>>>>>>>>>>>>>> other factors.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> We could always move this function over to using a
> > >>>>>>>>> KScatteredTable
> > >>>>>>>>>>>>>>>> implementation in the future, and simply deprecate it
> > >>>>> this
> > >>>>>>>> join
> > >>>>>>>>>> API
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>>>> time. I think I would like to hear more from some of the
> > >>>>>>> other
> > >>>>>>>>>> major
> > >>>>>>>>>>>>>>>> committers on which course of action they would think is
> > >>>>>>> best
> > >>>>>>>>>> before
> > >>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>> more coding is done.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks again
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> > >>>>>>>> john@confluent.io>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Hi Jan and Adam,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
> > >>>>> are
> > >>>>>>>>>>> encouraging.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
> > >>>>>>> agree
> > >>>>>>>>> that
> > >>>>>>>>>>>>>> avoiding
> > >>>>>>>>>>>>>>>>> unnecessary join outputs is especially important when
> > >>>>> the
> > >>>>>>>>> fan-out
> > >>>>>>>>>>> is
> > >>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>>> high. I suppose this could also be built into the
> > >>>>>>>>> implementation
> > >>>>>>>>>>>> we're
> > >>>>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
> > >>>>>>> KIP
> > >>>>>>>>>> (since
> > >>>>>>>>>>>>>> it's an
> > >>>>>>>>>>>>>>>>> API-transparent optimization).
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
> > >>>>>>> didn't
> > >>>>>>>>>> bring
> > >>>>>>>>>>>> it
> > >>>>>>>>>>>>>> up
> > >>>>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
> > >>>>> the
> > >>>>>>>> KIP
> > >>>>>>>>>>> as-is,
> > >>>>>>>>>>>>>>>>> despite the disagreement.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
> > >>>>>>>>>> reasonable.
> > >>>>>>>>>>>>>>>>> It sounds like Jan values more the potential for
> > >>>>>>> developers
> > >>>>>>>> to
> > >>>>>>>>>>>> optimize
> > >>>>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
> > >>>>> whereas
> > >>>>>>>> Adam
> > >>>>>>>>>>>> places
> > >>>>>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>> value on having a single operator that people can use
> > >>>>>>> without
> > >>>>>>>>>> extra
> > >>>>>>>>>>>>>> steps
> > >>>>>>>>>>>>>>>>> at the end.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Personally, although I do find it exceptionally
> > >>>>> annoying
> > >>>>>>>> when a
> > >>>>>>>>>>>>>> framework
> > >>>>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
> > >>>>> it
> > >>>>>>>> seems
> > >>>>>>>>>>> better
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> go
> > >>>>>>>>>>>>>>>>> for a single operation.
> > >>>>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
> > >>>>>>> significant
> > >>>>>>>>>>>> latitude
> > >>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>> the implementation (for example, joining only at the
> > >>>>> end,
> > >>>>>>> not
> > >>>>>>>>> in
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> middle
> > >>>>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
> > >>>>> resolution;
> > >>>>>>> how
> > >>>>>>>> we
> > >>>>>>>>>>>>>> represent
> > >>>>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
> > >>>>>>> vectors),
> > >>>>>>>>>>> etc.).
> > >>>>>>>>>>>>>> If we
> > >>>>>>>>>>>>>>>>> publish something like a KScatteredTable with the
> > >>>>>>>>>> right-partitioned
> > >>>>>>>>>>>>>> joined
> > >>>>>>>>>>>>>>>>> data, then the API pretty much locks in the
> > >>>>>>> implementation as
> > >>>>>>>>>> well.
> > >>>>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
> > >>>>> mean
> > >>>>>>>>> "seems";
> > >>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>> anyone
> > >>>>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
> > >>>>>>>>> simpler,
> > >>>>>>>>>> I
> > >>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
> > >>>>>>> algebra
> > >>>>>>>>>>>>>> perspective,
> > >>>>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
> > >>>>>>> KTable
> > >>>>>>>>> in
> > >>>>>>>>>>> all
> > >>>>>>>>>>>>>>>>> cases.
> > >>>>>>>>>>>>>>>>> * That said, there might still be room in the API for a
> > >>>>>>>>> different
> > >>>>>>>>>>>>>>>>> operation
> > >>>>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
> > >>>>> then
> > >>>>>>> do
> > >>>>>>>>>> things
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
> > >>>>> haven't
> > >>>>>>>>> thought
> > >>>>>>>>>>>>>> through
> > >>>>>>>>>>>>>>>>> all the consequences yet.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> This is all just my opinion after thinking over the
> > >>>>>>>> discussion
> > >>>>>>>>> so
> > >>>>>>>>>>>>>> far...
> > >>>>>>>>>>>>>>>>> -John
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > >>>>>>>>>>>>>> adam.bellemare@gmail.com>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
> > >>>>> the
> > >>>>>>>>>>>> prefixScan.
> > >>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>> have attached the file, but I will also include the
> > >>>>> text
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>> body
> > >>>>>>>>>>>>>>>>> here
> > >>>>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
> > >>>>>>>> attached
> > >>>>>>>>>>>> files).
> > >>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
> > >>>>> large
> > >>>>>>>>> values
> > >>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
> > >>>>>>> would
> > >>>>>>>> be
> > >>>>>>>>>> more
> > >>>>>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
> > >>>>>>> For
> > >>>>>>>>>>> instance,
> > >>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
> > >>>>> and a
> > >>>>>>>>> further
> > >>>>>>>>>>>>>>>>> ~35-85mS
> > >>>>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
> > >>>>>>> them.
> > >>>>>>>>>>> Iterating
> > >>>>>>>>>>>>>>>>>> through the records just to generate a simple count
> > >>>>>>> takes ~
> > >>>>>>>> 40
> > >>>>>>>>>>> times
> > >>>>>>>>>>>>>>>>> longer
> > >>>>>>>>>>>>>>>>>> than the flush + scan combined.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>>>>>>>>>>>>>> Setup:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
> > >>>>>>>>> (Xmx512m,
> > >>>>>>>>>>>>>> Xms512m)
> > >>>>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
> > >>>>> directly-accessible
> > >>>>>>>>> Kafka
> > >>>>>>>>>>>>>> Streams
> > >>>>>>>>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
> > >>>>> the
> > >>>>>>>>>>>>>>>>>> ProcessorContext).
> > >>>>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
> > >>>>> values
> > >>>>>>>>>> provided
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> 2.1/trunk release.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> keysize = 128 bytes
> > >>>>>>>>>>>>>>>>>> valsize = 512 bytes
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Step 1:
> > >>>>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
> > >>>>>>>> left-padded
> > >>>>>>>>>>>>>>>>>> auto-incrementing integer)
> > >>>>>>>>>>>>>>>>>> Step 2:
> > >>>>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
> > >>>>>>>>>>>>>> auto-incrementing
> > >>>>>>>>>>>>>>>>>> integer)
> > >>>>>>>>>>>>>>>>>> Step 3:
> > >>>>>>>>>>>>>>>>>> Perform flush
> > >>>>>>>>>>>>>>>>>> Step 4:
> > >>>>>>>>>>>>>>>>>> Perform prefixScan
> > >>>>>>>>>>>>>>>>>> Step 5:
> > >>>>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
> > >>>>> count of
> > >>>>>>>>>> expected
> > >>>>>>>>>>>>>>>>> events.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>>>>>>>>>>>>>> Results:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>>>>>>>>>>>>>> X = 1k (11k events total)
> > >>>>>>>>>>>>>>>>>> Flush Time = 39 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 7 mS
> > >>>>>>>>>>>>>>>>>> 6.9 MB disk
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>>>>>>>>>>>>> X = 10k (110k events total)
> > >>>>>>>>>>>>>>>>>> Flush Time = 45 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 8 mS
> > >>>>>>>>>>>>>>>>>> 127 MB
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>>>>>>>>>>>>> X = 100k (1.1M events total)
> > >>>>>>>>>>>>>>>>>> Test1:
> > >>>>>>>>>>>>>>>>>> Flush Time = 60 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 12 mS
> > >>>>>>>>>>>>>>>>>> 678 MB
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Test2:
> > >>>>>>>>>>>>>>>>>> Flush Time = 45 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 7 mS
> > >>>>>>>>>>>>>>>>>> 576 MB
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>>>>>>>>>>>>> X = 1MB (11M events total)
> > >>>>>>>>>>>>>>>>>> Test1:
> > >>>>>>>>>>>>>>>>>> Flush Time = 52 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 19 mS
> > >>>>>>>>>>>>>>>>>> 7.2 GB
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Test2:
> > >>>>>>>>>>>>>>>>>> Flush Time = 84 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 34 mS
> > >>>>>>>>>>>>>>>>>> 9.1 GB
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
> > >>>>>>>>>>>>>>>>>> Test1:
> > >>>>>>>>>>>>>>>>>> Flush Time = 82 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 63 mS
> > >>>>>>>>>>>>>>>>>> 17GB - 276 sst files
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Test2:
> > >>>>>>>>>>>>>>>>>> Flush Time = 116 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 35 mS
> > >>>>>>>>>>>>>>>>>> 23GB - 361 sst files
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Test3:
> > >>>>>>>>>>>>>>>>>> Flush Time = 103 mS
> > >>>>>>>>>>>>>>>>>> Scan Time = 82 mS
> > >>>>>>>>>>>>>>>>>> 19 GB - 300 sst files
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
> > >>>>>>> events. I
> > >>>>>>>>>> tried
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>> go
> > >>>>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
> > >>>>> the
> > >>>>>>>> 100GB+
> > >>>>>>>>>>> range
> > >>>>>>>>>>>>>>>>> and my
> > >>>>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
> > >>>>> be
> > >>>>>>> done
> > >>>>>>>>>> but I
> > >>>>>>>>>>>>>>>>> suspect
> > >>>>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
> > >>>>>>>> results
> > >>>>>>>>>>>> above.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> At this point in time, I think the only major
> > >>>>> discussion
> > >>>>>>>> point
> > >>>>>>>>>> is
> > >>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
> > >>>>> repartitioning
> > >>>>>>>> back +
> > >>>>>>>>>>>>>> resolving
> > >>>>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
> > >>>>> the
> > >>>>>>>> client
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> handle.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Thanks folks,
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > >>>>>>>>>>>>>>>>>>>> Hi all,
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
> > >>>>> the
> > >>>>>>> 2.1
> > >>>>>>>>>>> release
> > >>>>>>>>>>>>>>>>>>> caused an
> > >>>>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
> > >>>>>>> radar
> > >>>>>>>>>> (which
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
> > >>>>>>> some
> > >>>>>>>>>> extend
> > >>>>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
> > >>>>>>> keep) me
> > >>>>>>>>>>>>>>>>> preoccupied.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
> > >>>>> guess
> > >>>>>>> Jan
> > >>>>>>>>> was
> > >>>>>>>>>>> on
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> right
> > >>>>>>>>>>>>>>>>>>>> track!
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
> > >>>>> and
> > >>>>>>> the
> > >>>>>>>>>> prior
> > >>>>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> API Thoughts:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
> > >>>>>>>> many-to-one
> > >>>>>>>>>>> join.
> > >>>>>>>>>>>>>>>>> Could
> > >>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
> > >>>>>>> flip
> > >>>>>>>>> the
> > >>>>>>>>>>>> design
> > >>>>>>>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
> > >>>>> join
> > >>>>>>>>> type,
> > >>>>>>>>>>> and
> > >>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
> > >>>>>>>>> one-to-one
> > >>>>>>>>>>>> join.
> > >>>>>>>>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>> would work, of course, but it would be super
> > >>>>>>> inefficient
> > >>>>>>>>>>> compared
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>> simple rekey-and-join.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
> > >>>>>>>> specified
> > >>>>>>>>>>>> whether
> > >>>>>>>>>>>>>>>>>>> it's an
> > >>>>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
> > >>>>>>> join, as
> > >>>>>>>>>>>>>>>>> (neglecting
> > >>>>>>>>>>>>>>>>>>> IQ),
> > >>>>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
> > >>>>>>> it in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> ValueJoiner.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
> > >>>>> quite
> > >>>>>>>>> right.
> > >>>>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
> > >>>>>>>>> paradigms
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>>>>> play in
> > >>>>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
> > >>>>>>> three
> > >>>>>>>>>>>> Serialized
> > >>>>>>>>>>>>>>>>>>> args, I
> > >>>>>>>>>>>>>>>>>>>> think it would be better to have one that allows
> > >>>>>>>>> (optionally)
> > >>>>>>>>>>>>>> setting
> > >>>>>>>>>>>>>>>>>>> the 4
> > >>>>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
> > >>>>>>>>>>> Materialized.
> > >>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
> > >>>>>>> already
> > >>>>>>>>> be
> > >>>>>>>>>>>>>>>>> available
> > >>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
> > >>>>>>> config
> > >>>>>>>>>> might
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>> applicable.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
> > >>>>> joins
> > >>>>>>>>> don't
> > >>>>>>>>>>>> allow
> > >>>>>>>>>>>>>>>>>>> setting
> > >>>>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
> > >>>>>>> since
> > >>>>>>>>> the
> > >>>>>>>>>>>> rekey
> > >>>>>>>>>>>>>>>>>>>> operation needs to produce results that are
> > >>>>>>> co-partitioned
> > >>>>>>>>>> with
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> "other"
> > >>>>>>>>>>>>>>>>>>>> KTable.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
> > >>>>>>> didn't
> > >>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>> follow
> > >>>>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
> > >>>>>>>>>> "deserializing"
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> record
> > >>>>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
> > >>>>>>> think
> > >>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
> > >>>>>>> namespace
> > >>>>>>>> at
> > >>>>>>>>>>> source
> > >>>>>>>>>>>>>>>>> nodes
> > >>>>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
> > >>>>>>> practically
> > >>>>>>>> no
> > >>>>>>>>>>>>>> overhead.
> > >>>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
> > >>>>> API
> > >>>>>>>> change
> > >>>>>>>>>> wrt
> > >>>>>>>>>>>>>>>>> headers
> > >>>>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
> > >>>>> placed
> > >>>>>>> on
> > >>>>>>>>>> users'
> > >>>>>>>>>>>>>>>>>>> headers.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
> > >>>>> the
> > >>>>>>>>> header
> > >>>>>>>>>> at
> > >>>>>>>>>>>>>>>>> all...
> > >>>>>>>>>>>>>>>>>>>> stay tuned)
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
> > >>>>>>> table
> > >>>>>>>>>>> growing
> > >>>>>>>>>>>>>>>>>>> without
> > >>>>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
> > >>>>>>>>> implementing
> > >>>>>>>>>>> OCC
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
> > >>>>> the
> > >>>>>>>>> rekey
> > >>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
> > >>>>> changes.
> > >>>>>>> As
> > >>>>>>>>> such,
> > >>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
> > >>>>> offset in
> > >>>>>>>> the
> > >>>>>>>>>>> source
> > >>>>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
> > >>>>> the
> > >>>>>>> same
> > >>>>>>>>>>> number
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>> keys
> > >>>>>>>>>>>>>>>>>>>> as the source table at all times.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
> > >>>>>>> might
> > >>>>>>>> be
> > >>>>>>>>>>>> relevant
> > >>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
> > >>>>>>> timestamp
> > >>>>>>>>> in
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
> > >>>>> the
> > >>>>>>>>>> timestamp
> > >>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>> OCC
> > >>>>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
> > >>>>>>> want to
> > >>>>>>>>> use
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
> > >>>>> use a
> > >>>>>>>>> similar
> > >>>>>>>>>>>>>>>>>>> mechanism to
> > >>>>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
> > >>>>>>>> re-keyed
> > >>>>>>>>>>>> values,
> > >>>>>>>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>>>>> you could avoid a separate table.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
> > >>>>> long
> > >>>>>>>>> time,
> > >>>>>>>>>> so
> > >>>>>>>>>>>>>> I've
> > >>>>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
> > >>>>> if we
> > >>>>>>>> can
> > >>>>>>>>>>> avoid
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> HWM
> > >>>>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
> > >>>>> final
> > >>>>>>>> join
> > >>>>>>>>>>>>>>>>> instead...
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
> > >>>>> Letter
> > >>>>>>>> FK,
> > >>>>>>>>>>> (other
> > >>>>>>>>>>>>>>>>>>> data))
> > >>>>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Left table:
> > >>>>>>>>>>>>>>>>>>>> 1: (A, xyz)
> > >>>>>>>>>>>>>>>>>>>> 2: (B, asd)
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Right table:
> > >>>>>>>>>>>>>>>>>>>> A: EntityA
> > >>>>>>>>>>>>>>>>>>>> B: EntityB
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
> > >>>>>>> key,
> > >>>>>>>> but
> > >>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>> propagating the value at all..
> > >>>>>>>>>>>>>>>>>>>> Rekey table:
> > >>>>>>>>>>>>>>>>>>>> A-1: (dummy value)
> > >>>>>>>>>>>>>>>>>>>> B-2: (dummy value)
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
> > >>>>>>>>>>>>>>>>>>>> A-1: EntityA
> > >>>>>>>>>>>>>>>>>>>> B-2: EntityB
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Which gets rekeyed back:
> > >>>>>>>>>>>>>>>>>>>> 1: A, EntityA
> > >>>>>>>>>>>>>>>>>>>> 2: B, EntityB
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> And finally we do the actual join:
> > >>>>>>>>>>>>>>>>>>>> Result table:
> > >>>>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
> > >>>>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
> > >>>>>>>> opportunity
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> compare
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
> > >>>>>>> the
> > >>>>>>>>> right
> > >>>>>>>>>>>>>>>>> table. If
> > >>>>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
> > >>>>>>> must be
> > >>>>>>>>>>>> outdated.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
> > >>>>> xyz)
> > >>>>>>> gets
> > >>>>>>>>>>> updated
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> (1:
> > >>>>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
> > >>>>> whether
> > >>>>>>> the
> > >>>>>>>>>> final
> > >>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
> > >>>>>>>> algorithm
> > >>>>>>>>>>> above,
> > >>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
> > >>>>>>> (B,
> > >>>>>>>>> xyz),
> > >>>>>>>>>>> (B,
> > >>>>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
> > >>>>>>>>> information
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
> > >>>>>>> mentioned
> > >>>>>>>>>> often
> > >>>>>>>>>>> I
> > >>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
> > >>>>>>>>> automatically.
> > >>>>>>>>>> I
> > >>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>> this is the real performance eater ;)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
> > >>>>>>>>>> performance
> > >>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> range scans when records change in the right table.
> > >>>>>>> You've
> > >>>>>>>>>> said
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> you've
> > >>>>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
> > >>>>>>> for a
> > >>>>>>>>>>> while.
> > >>>>>>>>>>>>>> Can
> > >>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
> > >>>>>>> you've
> > >>>>>>>>>>>> observed?
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
> > >>>>>>> topmost
> > >>>>>>>>>> thing
> > >>>>>>>>>>>> here
> > >>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
> > >>>>>>>>> performance
> > >>>>>>>>>>> of
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
> > >>>>>>> 500k -
> > >>>>>>>>> 1kk
> > >>>>>>>>>>> rows
> > >>>>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
> > >>>>> the
> > >>>>>>> work
> > >>>>>>>>> you
> > >>>>>>>>>>>> gotta
> > >>>>>>>>>>>>>>>>>>> do, also when you pack your data into different
> > >>>>> formats,
> > >>>>>>>>>> usually
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
> > >>>>> data
> > >>>>>>> and
> > >>>>>>>>> we
> > >>>>>>>>>>>> can't
> > >>>>>>>>>>>>>>>>>>> really change that. It is more important for users to
> > >>>>>>>> prevent
> > >>>>>>>>>>>> useless
> > >>>>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
> > >>>>> to
> > >>>>>>> drop
> > >>>>>>>>>>> changes
> > >>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> are not going to change my join output.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> usually it's:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
> > >>>>>>>> old.equals(new)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
> > >>>>> for
> > >>>>>>>>>> smaller
> > >>>>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
> > >>>>> then
> > >>>>>>>>> anyways.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
> > >>>>>>> sure if
> > >>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>> better
> > >>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
> > >>>>> the
> > >>>>>>>>>> original
> > >>>>>>>>>>>>>> key,
> > >>>>>>>>>>>>>>>>>>> as I
> > >>>>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
> > >>>>> keys in
> > >>>>>>>> the
> > >>>>>>>>>>> value:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Left table:
> > >>>>>>>>>>>>>>>>>>>> 1: A,...
> > >>>>>>>>>>>>>>>>>>>> 2: B,...
> > >>>>>>>>>>>>>>>>>>>> 3: A,...
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Gets re-keyed:
> > >>>>>>>>>>>>>>>>>>>> A: [1, 3]
> > >>>>>>>>>>>>>>>>>>>> B: [2]
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
> > >>>>>>> regular
> > >>>>>>>>>>>> single-key
> > >>>>>>>>>>>>>>>>>>> lookup.
> > >>>>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
> > >>>>>>>> values,
> > >>>>>>>>> as
> > >>>>>>>>>>>>>>>>> there's
> > >>>>>>>>>>>>>>>>>>> no
> > >>>>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
> > >>>>> reference
> > >>>>>>> rhs
> > >>>>>>>>>>> records.
> > >>>>>>>>>>>>>>>>>>> Offhand,
> > >>>>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
> > >>>>>>> past
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> threshold, we
> > >>>>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
> > >>>>> cases,
> > >>>>>>> it
> > >>>>>>>>> would
> > >>>>>>>>>>> be
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> single
> > >>>>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
> > >>>>> be
> > >>>>>>> one
> > >>>>>>>>> per
> > >>>>>>>>>>>> (max
> > >>>>>>>>>>>>>>>>>>> value
> > >>>>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
> > >>>>>>> there's
> > >>>>>>>>> some
> > >>>>>>>>>>>> extra
> > >>>>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
> > >>>>>>>> retraction.
> > >>>>>>>>>> For
> > >>>>>>>>>>>>>>>>> example,
> > >>>>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
> > >>>>> just
> > >>>>>>> have
> > >>>>>>>>> (A:
> > >>>>>>>>>>>> [3]).
> > >>>>>>>>>>>>>>>>>>> Some
> > >>>>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
> > >>>>>>> for 1
> > >>>>>>>>> can
> > >>>>>>>>>>> also
> > >>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>> retracted.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> That's all!
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
> > >>>>> thoughtful
> > >>>>>>>> KIP.
> > >>>>>>>>>>> Sorry
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> discussion has been slow.
> > >>>>>>>>>>>>>>>>>>>> -John
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > >>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
> > >>>>> up,
> > >>>>>>> it
> > >>>>>>>>> just
> > >>>>>>>>>>>> goes
> > >>>>>>>>>>>>>>>>> back
> > >>>>>>>>>>>>>>>>>>>>> to discuss.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> would not expect to much attention with another
> > >>>>>>> another
> > >>>>>>>>> email
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>> thread.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> best Jan
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>>>>>>>>> Hello Contributors
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
> > >>>>>>> need
> > >>>>>>>> to
> > >>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>> this to
> > >>>>>>>>>>>>>>>>>>>>> keep
> > >>>>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
> > >>>>>>> through
> > >>>>>>>>>> once
> > >>>>>>>>>>>>>>>>>>> contributor
> > >>>>>>>>>>>>>>>>>>>>>> feedback is given.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Main points that need addressing:
> > >>>>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
> > >>>>>>>>> singular
> > >>>>>>>>>>>> graph
> > >>>>>>>>>>>>>>>>> node
> > >>>>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
> > >>>>> right
> > >>>>>>>>> now. I
> > >>>>>>>>>>> am
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>>>>>> fuzzy
> > >>>>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
> > >>>>> I
> > >>>>>>>> would
> > >>>>>>>>>>>>>>>>> appreciate
> > >>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>> help on this aspect.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
> > >>>>>>> thread
> > >>>>>>>>> has
> > >>>>>>>>>>>> much
> > >>>>>>>>>>>>>>>>>>>>> discourse
> > >>>>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
> > >>>>> mark
> > >>>>>>>>>> proposal
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>> groupBy
> > >>>>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
> > >>>>> need
> > >>>>>>> to
> > >>>>>>>>>>> strictly
> > >>>>>>>>>>>>>>>>> handle
> > >>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
> > >>>>> up
> > >>>>>>> to
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>>> consumer.
> > >>>>>>>>>>>>>>>>>>> Any
> > >>>>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
> > >>>>> this
> > >>>>>>>> from
> > >>>>>>>>>>>> moving
> > >>>>>>>>>>>>>>>>> to a
> > >>>>>>>>>>>>>>>>>>>>> vote?
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > >>>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Hi Jan
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
> > >>>>>>>>>>>>>>>>> Stores.persistentWindowStore,
> > >>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
> > >>>>> segments
> > >>>>>>>> you
> > >>>>>>>>>> want
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> how
> > >>>>>>>>>>>>>>>>>>>>> large
> > >>>>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
> > >>>>>>> happens
> > >>>>>>>>> is
> > >>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
> > >>>>> data
> > >>>>>>> with
> > >>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>> timestamps
> > >>>>>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
> > >>>>> of
> > >>>>>>> the
> > >>>>>>>>>> work
> > >>>>>>>>>>>> done
> > >>>>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
> > >>>>> the
> > >>>>>>>>>> hopping
> > >>>>>>>>>>>>>>>>> windowed
> > >>>>>>>>>>>>>>>>>>>>> store.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
> > >>>>>>> to be
> > >>>>>>>>> two
> > >>>>>>>>>>> 24h
> > >>>>>>>>>>>>>>>>>>>>> segments.
> > >>>>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
> > >>>>> will
> > >>>>>>> be
> > >>>>>>>>>>> inserted
> > >>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>> T in
> > >>>>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
> > >>>>> overlap
> > >>>>>>> by
> > >>>>>>>>> 12h.
> > >>>>>>>>>>> As
> > >>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>> goes on
> > >>>>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
> > >>>>> the
> > >>>>>>>>> oldest
> > >>>>>>>>>>>>>> segment
> > >>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
> > >>>>> The
> > >>>>>>>>>> records
> > >>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>> default
> > >>>>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
> > >>>>> it
> > >>>>>>> is
> > >>>>>>>>> the
> > >>>>>>>>>>>> record
> > >>>>>>>>>>>>>>>>>>> time,
> > >>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
> > >>>>> are
> > >>>>>>>>>> retained
> > >>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
> > >>>>> TTL
> > >>>>>>>> at a
> > >>>>>>>>>>>>>>>>>>> segment-level,
> > >>>>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> > >>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
> > >>>>>>>>>>>> ClassCastException
> > >>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>> similar.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
> > >>>>>>>>> fetch/put
> > >>>>>>>>>> or
> > >>>>>>>>>>>>>>>>> iterate
> > >>>>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
> > >>>>>>> stream-time
> > >>>>>>>>>> under
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> hoods
> > >>>>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
> > >>>>> all
> > >>>>>>> the
> > >>>>>>>>>>> windows
> > >>>>>>>>>>>>>>>>>>> earlier?
> > >>>>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
> > >>>>>>> time
> > >>>>>>>>>>>> gurantees.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Best
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
> > >>>>>>> changed
> > >>>>>>>>> the
> > >>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>> store,
> > >>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
> > >>>>>>>> something
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> will be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> extremely
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
> > >>>>> said, I
> > >>>>>>>> will
> > >>>>>>>>>> not
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>> moving
> > >>>>>>>>>>>>>>>>>>>>>>>> ahead
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
> > >>>>> reshuffle/groupBy
> > >>>>>>>>> solution
> > >>>>>>>>>>> as
> > >>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>> propose.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
> > >>>>>>>> yourself
> > >>>>>>>>>> off
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>>>> current PR
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
> > >>>>>>> would
> > >>>>>>>>> be
> > >>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>>>> happy to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> > >>>>>>>> stands
> > >>>>>>>>>>> right
> > >>>>>>>>>>>>>>>>> now,
> > >>>>>>>>>>>>>>>>>>> I do
> > >>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
> > >>>>>>> alternatives
> > >>>>>>>>>> without
> > >>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
> > >>>>> which
> > >>>>>>>> they
> > >>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>> prefer.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
> > >>>>> gave
> > >>>>>>> up
> > >>>>>>>> on
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> streams
> > >>>>>>>>>>>>>>>>>>>>> DSL
> > >>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
> > >>>>> pulled
> > >>>>>>>> this
> > >>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>> through
> > >>>>>>>>>>>>>>>>>>>>>>>> already.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
> > >>>>> based on
> > >>>>>>>>> PAPI.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
> > >>>>>>> windowed
> > >>>>>>>>>> state
> > >>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
> > >>>>> then I
> > >>>>>>>> will
> > >>>>>>>>>> come
> > >>>>>>>>>>>>>> back
> > >>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>> final
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
> > >>>>> any of
> > >>>>>>>> the
> > >>>>>>>>>>>> binding
> > >>>>>>>>>>>>>>>>>>> voters
> > >>>>>>>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
> > >>>>>>> updated
> > >>>>>>>> it
> > >>>>>>>>>>>>>>>>> according
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
> > >>>>> windowed
> > >>>>>>>>> store.
> > >>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
> > >>>>> they
> > >>>>>>>> are
> > >>>>>>>>>>>>>>>>> completed.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> > >>>>>>>> already
> > >>>>>>>>>>>> updated
> > >>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> PR?
> > >>>>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
> > >>>>> Missing
> > >>>>>>>>>>> something?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
> > >>>>> Wang <
> > >>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
> > >>>>> is
> > >>>>>>> the
> > >>>>>>>>>> wrong
> > >>>>>>>>>>>>>> link,
> > >>>>>>>>>>>>>>>>>>> as it
> > >>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
> > >>>>>>> part of
> > >>>>>>>>>>> KIP-258
> > >>>>>>>>>>>>>>>>> we do
> > >>>>>>>>>>>>>>>>>>>>>>>> want to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
> > >>>>>>>> KTable"
> > >>>>>>>>>> such
> > >>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>> instead of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
> > >>>>> materialized
> > >>>>>>>> store,
> > >>>>>>>>>>> i.e.
> > >>>>>>>>>>>>>>>>>>> following
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
> > >>>>> older
> > >>>>>>>> than
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>>>> key's
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
> > >>>>> ordering.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
> > >>>>>>> Wang <
> > >>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
> > >>>>>>> final
> > >>>>>>>>> step
> > >>>>>>>>>>>> (i.e.
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> high
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
> > >>>>>>> with
> > >>>>>>>> a
> > >>>>>>>>>>> window
> > >>>>>>>>>>>>>>>>>>> store),
> > >>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
> > >>>>>>> help:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
> > >>>>>>> key-value
> > >>>>>>>>>> store
> > >>>>>>>>>>>>>>>>> (i.e.
> > >>>>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
> > >>>>>>> usage,
> > >>>>>>>> as
> > >>>>>>>>>>>>>>>>> described
> > >>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
> > >>>>>>>> ,
> > >>>>>>>>> is
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
> > >>>>> its
> > >>>>>>>>>>> timestamp
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>> smaller
> > >>>>>>>>>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
> > >>>>> I
> > >>>>>>>> think
> > >>>>>>>>> it
> > >>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>> very
> > >>>>>>>>>>>>>>>>>>>>>>>> similar to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
> > >>>>> based
> > >>>>>>>>>>> filtering,
> > >>>>>>>>>>>>>>>>> while
> > >>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
> > >>>>> the
> > >>>>>>>>> joining
> > >>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
> > >>>>>>> final
> > >>>>>>>>>> stage.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
> > >>>>> and
> > >>>>>>>> hence
> > >>>>>>>>>>>>>>>>>>> non-windowed
> > >>>>>>>>>>>>>>>>>>>>>>>> KTables
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
> > >>>>>>> really
> > >>>>>>>>> have
> > >>>>>>>>>> a
> > >>>>>>>>>>>> good
> > >>>>>>>>>>>>>>>>>>>>> support
> > >>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
> > >>>>>>>>> KTable-KTable
> > >>>>>>>>>>>>>>>>> non-key
> > >>>>>>>>>>>>>>>>>>>>> joins
> > >>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
> > >>>>> Filipiak
> > >>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
> > >>>>> would
> > >>>>>>>> grow
> > >>>>>>>>>>>>>> endlessly
> > >>>>>>>>>>>>>>>>>>> based
> > >>>>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
> > >>>>> pair
> > >>>>>>> of
> > >>>>>>>>>> (<this
> > >>>>>>>>>>>>>>>>> table
> > >>>>>>>>>>>>>>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
> > >>>>> is
> > >>>>>>> used
> > >>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> differentiate
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> between
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
> > >>>>>>> proposal
> > >>>>>>>>>> would
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> replace
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
> > >>>>>>> This
> > >>>>>>>>> would
> > >>>>>>>>>>>> allow
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
> > >>>>> time.
> > >>>>>>> This
> > >>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>> allow
> > >>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>> all
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> > >>>>>>>> should
> > >>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>> customizable
> > >>>>>>>>>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
> > >>>>>>> perhaps
> > >>>>>>>>> just
> > >>>>>>>>>>> 10
> > >>>>>>>>>>>>>>>>>>> minutes
> > >>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
> > >>>>> do
> > >>>>>>> the
> > >>>>>>>>>> trick
> > >>>>>>>>>>>>>> here.
> > >>>>>>>>>>>>>>>>>>> Even
> > >>>>>>>>>>>>>>>>>>>>>>>> if I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
> > >>>>>>>>> repartitioning
> > >>>>>>>>>>>>>>>>> optional
> > >>>>>>>>>>>>>>>>>>>>>>>> since I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
> > >>>>>>> am a
> > >>>>>>>>>> little
> > >>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>>>>>>>>> sceptical
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
> > >>>>> one
> > >>>>>>>>> could
> > >>>>>>>>>>> run
> > >>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>>>> problems
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
> > >>>>>>> border. I
> > >>>>>>>>> will
> > >>>>>>>>>>>> check
> > >>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
> > >>>>>>> problematic, we
> > >>>>>>>>>> could
> > >>>>>>>>>>>>>>>>> still
> > >>>>>>>>>>>>>>>>>>>>> check
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
> > >>>>> performance
> > >>>>>>>>> impact I
> > >>>>>>>>>>>>>> guess.
> > >>>>>>>>>>>>>>>>>>> Will
> > >>>>>>>>>>>>>>>>>>>>>>>> let
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
> > >>>>> correct
> > >>>>>>> as
> > >>>>>>>>> is. I
> > >>>>>>>>>>>>>>>>> wouldn't
> > >>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> > >>>>>>>>> timestamp(A)  <
> > >>>>>>>>>>>>>>>>>>>>> timestamp(B).
> > >>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
> > >>>>> -
> > >>>>>>>> thanks
> > >>>>>>>>>> for
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> diagram, it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
> > >>>>> do
> > >>>>>>> not
> > >>>>>>>>> have
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
> > >>>>>>>>> available
> > >>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>> would be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
> > >>>>> Map.
> > >>>>>>>> That
> > >>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>> said,
> > >>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
> > >>>>> for
> > >>>>>>>>> clarity
> > >>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>> everyone
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
> > >>>>> really
> > >>>>>>> hard
> > >>>>>>>>>> work.
> > >>>>>>>>>>>> But
> > >>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> > >>>>>>>>> original
> > >>>>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>>>> key,
> > >>>>>>>>>>>>>>>>>>>>> We
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
> > >>>>> PAPI
> > >>>>>>>> and
> > >>>>>>>>>>>>>> basically
> > >>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
> > >>>>> missed
> > >>>>>>>> that
> > >>>>>>>>> in
> > >>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>> DSL
> > >>>>>>>>>>>>>>>>>>>>>>>> its
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
> > >>>>>>> up on
> > >>>>>>>>> my
> > >>>>>>>>>>> end.
> > >>>>>>>>>>>>>>>>> Will
> > >>>>>>>>>>>>>>>>>>>>>>>> finish
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
> > >>>>>>> week.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
> > >>>>> the
> > >>>>>>> Map
> > >>>>>>>>> stay
> > >>>>>>>>>>>>>> inside
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> State
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
> > >>>>> changes
> > >>>>>>>> have
> > >>>>>>>>>>>>>>>>> propagated?
> > >>>>>>>>>>>>>>>>>>>>> Isn't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
> > >>>>>>> state
> > >>>>>>>>>> store?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
> > >>>>>>> substractor
> > >>>>>>>> is
> > >>>>>>>>>>> gonna
> > >>>>>>>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>>>>>>>>>> `null`
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
> > >>>>>>> there
> > >>>>>>>> is
> > >>>>>>>>>>> going
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
> > >>>>> this
> > >>>>>>>> store
> > >>>>>>>>>>>> directly
> > >>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
> > >>>>> is a
> > >>>>>>>>>> regular
> > >>>>>>>>>>>>>>>>> store,
> > >>>>>>>>>>>>>>>>>>>>>>>> satisfying
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
> > >>>>>>> join.
> > >>>>>>>>> The
> > >>>>>>>>>>>>>>>>> Windowed
> > >>>>>>>>>>>>>>>>>>>>>>>> store is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
> > >>>>> statefull
> > >>>>>>>>>> operation
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
> > >>>>>>> have
> > >>>>>>>> the
> > >>>>>>>>>>>> window
> > >>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
> > >>>>> custom
> > >>>>>>>> group
> > >>>>>>>>>> by
> > >>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
> > >>>>> key i
> > >>>>>>>> think
> > >>>>>>>>>> it
> > >>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>> help
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> users
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
> > >>>>> the
> > >>>>>>>>>> original
> > >>>>>>>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
> > >>>>>>> foundation
> > >>>>>>>>> to
> > >>>>>>>>>>>> build
> > >>>>>>>>>>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
> > >>>>> user.
> > >>>>>>>> very
> > >>>>>>>>>>>>>>>>>>> unfortunate. I
> > >>>>>>>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
> > >>>>> do
> > >>>>>>> not
> > >>>>>>>>>> think
> > >>>>>>>>>>>> its
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> good
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
> > >>>>> Prajakta
> > >>>>>>>>> Dumbre <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please remove me from this
> > >>>>> group
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             On Tue, Sep 11, 2018 at 1:29
> PM
> > >>>>>>> Jan
> > >>>>>>>>>>> Filipiak
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> > >>>>>>> <mailto:
> > >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hi Adam,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > give me some time, will
> make
> > >>>>>>> such a
> > >>>>>>>>>>> chart.
> > >>>>>>>>>>>>>> last
> > >>>>>>>>>>>>>>>>>>> time i
> > >>>>>>>>>>>>>>>>>>>>>>>> didn't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             get along
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > well with giphy and ruined
> > >>>>> all
> > >>>>>>> your
> > >>>>>>>>>>> charts.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hopefully i can get it done
> > >>>>>>> today
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > On 08.09.2018 16:00, Adam
> > >>>>>>> Bellemare
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Hi Jan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I have included a diagram
> > >>>>> of
> > >>>>>>>> what I
> > >>>>>>>>>>>>>> attempted
> > >>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I attempted this back at
> > >>>>> the
> > >>>>>>>> start
> > >>>>>>>>> of
> > >>>>>>>>>>> my
> > >>>>>>>>>>>> own
> > >>>>>>>>>>>>>>>>>>>>>>>> implementation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > solution, and since I
> could
> > >>>>>>> not
> > >>>>>>>> get
> > >>>>>>>>>> it
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> work I
> > >>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             discarded the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > code. At this point in
> > >>>>> time,
> > >>>>>>> if
> > >>>>>>>> you
> > >>>>>>>>>>> wish
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> continue
> > >>>>>>>>>>>>>>>>>>>>>>>> pursuing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             for your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > groupBy solution, I ask
> > >>>>> that
> > >>>>>>> you
> > >>>>>>>>>> please
> > >>>>>>>>>>>>>>>>> create a
> > >>>>>>>>>>>>>>>>>>>>>>>> diagram on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the KIP
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > carefully explaining your
> > >>>>>>>> solution.
> > >>>>>>>>>>>> Please
> > >>>>>>>>>>>>>>>>> feel
> > >>>>>>>>>>>>>>>>>>> free
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the image I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > just posted as a starting
> > >>>>>>> point.
> > >>>>>>>> I
> > >>>>>>>>> am
> > >>>>>>>>>>>> having
> > >>>>>>>>>>>>>>>>>>> trouble
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             understanding your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > explanations but I think
> > >>>>> that
> > >>>>>>> a
> > >>>>>>>>>>> carefully
> > >>>>>>>>>>>>>>>>>>> constructed
> > >>>>>>>>>>>>>>>>>>>>>>>> diagram
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             will clear
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > up
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > any misunderstandings.
> > >>>>>>>> Alternately,
> > >>>>>>>>>>>> please
> > >>>>>>>>>>>>>>>>> post a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             comprehensive PR with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > your solution. I can only
> > >>>>>>> guess
> > >>>>>>>> at
> > >>>>>>>>>> what
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>> mean, and
> > >>>>>>>>>>>>>>>>>>>>>>>> since I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             value my
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > own
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > time as much as you value
> > >>>>>>> yours,
> > >>>>>>>> I
> > >>>>>>>>>>>> believe
> > >>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             responsibility to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > provide an implementation
> > >>>>>>> instead
> > >>>>>>>>> of
> > >>>>>>>>>> me
> > >>>>>>>>>>>>>>>>> trying to
> > >>>>>>>>>>>>>>>>>>>>> guess.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > On Sat, Sep 8, 2018 at
> 8:00
> > >>>>>>> AM,
> > >>>>>>>> Jan
> > >>>>>>>>>>>> Filipiak
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> > >>>>>>> <mailto:
> > >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Hi James,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> nice to see you beeing
> > >>>>>>>> interested.
> > >>>>>>>>>>> kafka
> > >>>>>>>>>>>>>>>>>>> streams at
> > >>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             point supports
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> all sorts of joins as
> > >>>>> long as
> > >>>>>>>> both
> > >>>>>>>>>>>> streams
> > >>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Adam is currently
> > >>>>>>> implementing a
> > >>>>>>>>>> join
> > >>>>>>>>>>>>>> where a
> > >>>>>>>>>>>>>>>>>>> KTable
> > >>>>>>>>>>>>>>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             KTable can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> a one to many relation
> > >>>>> ship
> > >>>>>>>> (1:n).
> > >>>>>>>>>> We
> > >>>>>>>>>>>>>> exploit
> > >>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>> rocksdb
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> datastore that keeps data
> > >>>>>>> sorted
> > >>>>>>>> (At
> > >>>>>>>>>>> least
> > >>>>>>>>>>>>>>>>>>> exposes an
> > >>>>>>>>>>>>>>>>>>>>>>>> API to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             access the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> stored data in a sorted
> > >>>>>>>> fashion).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I think the technical
> > >>>>> caveats
> > >>>>>>>> are
> > >>>>>>>>>> well
> > >>>>>>>>>>>>>>>>>>> understood
> > >>>>>>>>>>>>>>>>>>>>> now
> > >>>>>>>>>>>>>>>>>>>>>>>> and we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > basically
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> down to philosophy and
> API
> > >>>>>>>> Design
> > >>>>>>>>> (
> > >>>>>>>>>>> when
> > >>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>> sees
> > >>>>>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>>>> newest
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             message).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I have a lengthy track
> > >>>>>>> record of
> > >>>>>>>>>>> loosing
> > >>>>>>>>>>>>>>>>> those
> > >>>>>>>>>>>>>>>>>>> kinda
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arguments within
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> streams community and I
> > >>>>> have
> > >>>>>>> no
> > >>>>>>>>> clue
> > >>>>>>>>>>>> why.
> > >>>>>>>>>>>>>> So
> > >>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>> literally
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             can't wait for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> to churn through this
> > >>>>> thread
> > >>>>>>> and
> > >>>>>>>>>> give
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>> opinion on
> > >>>>>>>>>>>>>>>>>>>>>>>> how we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             should
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > design
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the return type of the
> > >>>>>>>>> oneToManyJoin
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>>> how
> > >>>>>>>>>>>>>>>>>>> many
> > >>>>>>>>>>>>>>>>>>>>>>>> power we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             want to give
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the user vs "simplicity"
> > >>>>>>> (where
> > >>>>>>>>>>>> simplicity
> > >>>>>>>>>>>>>>>>> isn't
> > >>>>>>>>>>>>>>>>>>>>>>>> really that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as users
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > still
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> need to understand it I
> > >>>>>>> argue)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> waiting for you to join
> > >>>>> in on
> > >>>>>>>> the
> > >>>>>>>>>>>>>> discussion
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Best Jan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> On 07.09.2018 15:49,
> James
> > >>>>>>> Kwan
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I am new to this group
> > >>>>> and I
> > >>>>>>>>> found
> > >>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>> subject
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             interesting.  Sounds
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> you guys want to
> > >>>>> implement a
> > >>>>>>>> join
> > >>>>>>>>>>>> table of
> > >>>>>>>>>>>>>>>>> two
> > >>>>>>>>>>>>>>>>>>>>>>>> streams? Is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > somewhere
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I can see the original
> > >>>>>>>>> requirement
> > >>>>>>>>>> or
> > >>>>>>>>>>>>>>>>> proposal?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> On Sep 7, 2018, at 8:13
> > >>>>> AM,
> > >>>>>>> Jan
> > >>>>>>>>>>>> Filipiak
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> > >>>>>>> <mailto:
> > >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> On 05.09.2018 22:17,
> > >>>>> Adam
> > >>>>>>>>>> Bellemare
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> I'm currently testing
> > >>>>>>> using a
> > >>>>>>>>>>>> Windowed
> > >>>>>>>>>>>>>>>>> Store
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> store the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             highwater
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> mark.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> By all indications
> this
> > >>>>>>>> should
> > >>>>>>>>>> work
> > >>>>>>>>>>>>>> fine,
> > >>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> caveat
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             being that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> only resolve
> > >>>>> out-of-order
> > >>>>>>>>> arrival
> > >>>>>>>>>>>> for up
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> size of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the window
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > (ie:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> 24h, 72h, etc). This
> > >>>>> would
> > >>>>>>>>> remove
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> possibility
> > >>>>>>>>>>>>>>>>>>>>>>>> of it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > unbounded
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> size.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> With regards to Jan's
> > >>>>>>>>>> suggestion, I
> > >>>>>>>>>>>>>>>>> believe
> > >>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>> where
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> remain in
> disagreement.
> > >>>>>>>> While I
> > >>>>>>>>>> do
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>> disagree
> > >>>>>>>>>>>>>>>>>>>>>>>> with your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             statement
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> about
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> there likely to be
> > >>>>>>> additional
> > >>>>>>>>>> joins
> > >>>>>>>>>>>> done
> > >>>>>>>>>>>>>>>>> in a
> > >>>>>>>>>>>>>>>>>>>>>>>> real-world
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             workflow, I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > do
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> see how you can
> > >>>>>>> conclusively
> > >>>>>>>>> deal
> > >>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> foreign-key
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> changes and
> subsequent
> > >>>>>>>> joins. I
> > >>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>> attempted
> > >>>>>>>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             think you have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> proposed (without a
> > >>>>>>>> high-water,
> > >>>>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>> groupBy and
> > >>>>>>>>>>>>>>>>>>>>>>>> reduce)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             and found
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> that if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> the foreign key
> changes
> > >>>>>>> too
> > >>>>>>>>>>> quickly,
> > >>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> load
> > >>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             stream thread
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> too
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> high, the joined
> > >>>>> messages
> > >>>>>>>> will
> > >>>>>>>>>>> arrive
> > >>>>>>>>>>>>>>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>>>>>>> and be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             incorrectly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> propagated, such that
> > >>>>> an
> > >>>>>>>>>>> intermediate
> > >>>>>>>>>>>>>>>>> event
> > >>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > final
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> event.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Can you shed some
> light
> > >>>>> on
> > >>>>>>>> your
> > >>>>>>>>>>>> groupBy
> > >>>>>>>>>>>>>>>>>>>>>>>> implementation.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             There must be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> some sort of flaw in
> it.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I have a suspicion
> > >>>>> where it
> > >>>>>>>> is,
> > >>>>>>>>> I
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             confirm. The idea
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> is bullet proof and it
> > >>>>>>> must be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> an implementation mess
> > >>>>> up.
> > >>>>>>> I
> > >>>>>>>>> would
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> clarify
> > >>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we draw a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> conclusion.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>    Repartitioning the
> > >>>>>>>> scattered
> > >>>>>>>>>>> events
> > >>>>>>>>>>>>>>>>> back to
> > >>>>>>>>>>>>>>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> partitions is the only
> > >>>>> way I
> > >>>>>>>> know
> > >>>>>>>>>> how
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> conclusively
> > >>>>>>>>>>>>>>>>>>>>>>>> deal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> out-of-order events
> in
> > >>>>> a
> > >>>>>>>> given
> > >>>>>>>>>> time
> > >>>>>>>>>>>>>> frame,
> > >>>>>>>>>>>>>>>>>>> and to
> > >>>>>>>>>>>>>>>>>>>>>>>> ensure
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > data
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> eventually consistent
> > >>>>> with
> > >>>>>>>> the
> > >>>>>>>>>>> input
> > >>>>>>>>>>>>>>>>> events.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> If you have some code
> > >>>>> to
> > >>>>>>>> share
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> illustrates
> > >>>>>>>>>>>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             approach, I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> very grateful as it
> > >>>>> would
> > >>>>>>>>> remove
> > >>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>> misunderstandings
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that I may
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> ah okay you were
> looking
> > >>>>>>> for
> > >>>>>>>> my
> > >>>>>>>>>>> code.
> > >>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             something easily
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> readable here as its
> > >>>>>>> bloated
> > >>>>>>>>> with
> > >>>>>>>>>>>>>>>>> OO-patterns.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> its anyhow trivial:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> @Override
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      public T apply(K
> > >>>>>>> aggKey,
> > >>>>>>>> V
> > >>>>>>>>>>>> value, T
> > >>>>>>>>>>>>>>>>>>>>> aggregate)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          Map<U, V>
> > >>>>>>>>>>> currentStateAsMap =
> > >>>>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             imaginary
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          U
> toModifyKey =
> > >>>>>>>>>>>>>>>>> mapper.apply(value);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << this
> is
> > >>>>> the
> > >>>>>>>>> place
> > >>>>>>>>>>>> where
> > >>>>>>>>>>>>>>>>> people
> > >>>>>>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             gonna have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > issues
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and why you probably
> > >>>>>>> couldn't
> > >>>>>>>> do
> > >>>>>>>>>> it.
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>> to find
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             a solution
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > here.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I didn't realize that
> > >>>>> yet.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << we
> > >>>>>>> propagate
> > >>>>>>>> the
> > >>>>>>>>>>>> field in
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> joiner, so
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that we can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > pick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> it up in an aggregate.
> > >>>>>>>> Probably
> > >>>>>>>>>> you
> > >>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>> thought
> > >>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this in your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> approach right?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I am
> > >>>>> very
> > >>>>>>> open
> > >>>>>>>>> to
> > >>>>>>>>>>>> find a
> > >>>>>>>>>>>>>>>>>>> generic
> > >>>>>>>>>>>>>>>>>>>>>>>> solution
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             here. In my
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> honest opinion this is
> > >>>>>>> broken
> > >>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
> > >>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             looses
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the keys
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and only maintains the
> > >>>>>>>> aggregate
> > >>>>>>>>>>> key.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I
> > >>>>>>> abstracted
> > >>>>>>>> it
> > >>>>>>>>>> away
> > >>>>>>>>>>>> back
> > >>>>>>>>>>>>>>>>>>> then way
> > >>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > thinking
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> of oneToMany join.
> That
> > >>>>> is
> > >>>>>>>> why I
> > >>>>>>>>>>>> didn't
> > >>>>>>>>>>>>>>>>>>> realize
> > >>>>>>>>>>>>>>>>>>>>> its
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             significance here.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              <<
> > >>>>> Opinions?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          for (V m :
> > >>>>>>> current)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>> currentStateAsMap.put(mapper.apply(m),
> > >>>>>>>>>>>>>> m);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          if (isAdder)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>> currentStateAsMap.put(toModifyKey,
> > >>>>>>>>>>>>>> value);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          else
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>> currentStateAsMap.remove(toModifyKey);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>> if(currentStateAsMap.isEmpty()){
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> return
> > >>>>>>> null;
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          retrun
> > >>>>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> On Wed, Sep 5, 2018
> at
> > >>>>>>> 3:35
> > >>>>>>>> PM,
> > >>>>>>>>>> Jan
> > >>>>>>>>>>>>>>>>> Filipiak
> > >>>>>>>>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Jan.Filipiak@trivago.com
> > >>>>>>> <mailto:
> > >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Thanks Adam for
> > >>>>> bringing
> > >>>>>>>>> Matthias
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> speed!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> about the
> > >>>>> differences. I
> > >>>>>>>> think
> > >>>>>>>>>>>>>> re-keying
> > >>>>>>>>>>>>>>>>>>> back
> > >>>>>>>>>>>>>>>>>>>>>>>> should be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             optional at
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> best.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I would say we
> return
> > >>>>> a
> > >>>>>>>>>>>> KScatteredTable
> > >>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>> reshuffle()
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             returning
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>> KTable<originalKey,Joined>
> > >>>>>>>> to
> > >>>>>>>>>> make
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> backwards
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartitioning
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> optional.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I am also in a big
> > >>>>>>> favour of
> > >>>>>>>>>> doing
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>> order
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             processing using
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> group
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by instead high
> water
> > >>>>>>> mark
> > >>>>>>>>>>> tracking.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Just because
> unbounded
> > >>>>>>>> growth
> > >>>>>>>>> is
> > >>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>> scary
> > >>>>>>>>>>>>>>>>>>> + It
> > >>>>>>>>>>>>>>>>>>>>>>>> saves
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the header
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> stuff.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I think the
> > >>>>> abstraction
> > >>>>>>> of
> > >>>>>>>>>> always
> > >>>>>>>>>>>>>>>>>>> repartitioning
> > >>>>>>>>>>>>>>>>>>>>>>>> back is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             just not so
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> strong. Like the
> work
> > >>>>> has
> > >>>>>>>> been
> > >>>>>>>>>>> done
> > >>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             back and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> grouping
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by something else
> > >>>>>>> afterwards
> > >>>>>>>>> is
> > >>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>>>> common.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> On 05.09.2018 13:49,
> > >>>>> Adam
> > >>>>>>>>>>> Bellemare
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Hi Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thank you for your
> > >>>>>>>> feedback,
> > >>>>>>>>> I
> > >>>>>>>>>> do
> > >>>>>>>>>>>>>>>>>>> appreciate
> > >>>>>>>>>>>>>>>>>>>>> it!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> While name spacing
> > >>>>>>> would be
> > >>>>>>>>>>>> possible,
> > >>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > deserialize
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
> > >>>>>>> implies
> > >>>>>>>> a
> > >>>>>>>>>>>> runtime
> > >>>>>>>>>>>>>>>>>>> overhead.
> > >>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now
> to
> > >>>>>>> avoid
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>> overhead.
> > >>>>>>>>>>>>>>>>>>> If
> > >>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
> > >>>>>>> still
> > >>>>>>>> add
> > >>>>>>>>>>> name
> > >>>>>>>>>>>>>>>>> spacing
> > >>>>>>>>>>>>>>>>>>>>> later
> > >>>>>>>>>>>>>>>>>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Agreed. I will go
> > >>>>> with
> > >>>>>>>>> using a
> > >>>>>>>>>>>>>> reserved
> > >>>>>>>>>>>>>>>>>>> string
> > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             document it.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> My main concern
> about
> > >>>>>>> the
> > >>>>>>>>>> design
> > >>>>>>>>>>> it
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> type of
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> understood the
> > >>>>> proposal
> > >>>>>>>>>>> correctly,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> In your example,
> you
> > >>>>>>> have
> > >>>>>>>>>> table1
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> table2
> > >>>>>>>>>>>>>>>>>>>>>>>> swapped.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Here is how it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> works
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> currently:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 1) table1 has the
> > >>>>>>> records
> > >>>>>>>>> that
> > >>>>>>>>>>>> contain
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> foreign key
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             within their
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> value.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 input
> stream:
> > >>>>>>>>>>>> <a,(fk=A,bar=1)>,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> <c,(fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table2 input
> stream:
> > >>>>>>> <A,X>,
> > >>>>>>>>>> <B,Y>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 2) A Value mapper
> is
> > >>>>>>>> required
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>> extract
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> foreign
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 foreign key
> > >>>>>>> mapper:
> > >>>>>>>> (
> > >>>>>>>>>>> value
> > >>>>>>>>>>>> =>
> > >>>>>>>>>>>>>>>>>>> value.fk
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <http://value.fk> )
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> The mapper is
> > >>>>> applied to
> > >>>>>>>> each
> > >>>>>>>>>>>> element
> > >>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> table1,
> > >>>>>>>>>>>>>>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             new combined
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> key is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> made:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 mapped:
> <A-a,
> > >>>>>>>>>>> (fk=A,bar=1)>,
> > >>>>>>>>>>>>>>>>> <A-b,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <B-c,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> (fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 3) The rekeyed
> events
> > >>>>>>> are
> > >>>>>>>>>>>>>> copartitioned
> > >>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>> table2:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> a) Stream Thread
> with
> > >>>>>>>>> Partition
> > >>>>>>>>>>> 0:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> RepartitionedTable1:
> > >>>>>>> <A-a,
> > >>>>>>>>>>>>>>>>> (fk=A,bar=1)>,
> > >>>>>>>>>>>>>>>>>>> <A-b,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             (fk=A,bar=2)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <A,X>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> b) Stream Thread
> with
> > >>>>>>>>> Partition
> > >>>>>>>>>>> 1:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> RepartitionedTable1:
> > >>>>>>> <B-c,
> > >>>>>>>>>>>>>> (fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <B,Y>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 4) From here, they
> > >>>>> can
> > >>>>>>> be
> > >>>>>>>>>> joined
> > >>>>>>>>>>>>>>>>> together
> > >>>>>>>>>>>>>>>>>>>>> locally
> > >>>>>>>>>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             applying the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> joiner
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> At this point,
> Jan's
> > >>>>>>> design
> > >>>>>>>>> and
> > >>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>> design
> > >>>>>>>>>>>>>>>>>>>>>>>> deviate. My
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design goes
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> repartition the
> data
> > >>>>>>>>> post-join
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> resolve
> > >>>>>>>>>>>>>>>>>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arrival of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> records,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> finally returning
> the
> > >>>>>>> data
> > >>>>>>>>>> keyed
> > >>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> original key.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             I do not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> expose
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> CombinedKey or any
> of
> > >>>>>>> the
> > >>>>>>>>>>> internals
> > >>>>>>>>>>>>>>>>>>> outside of
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             joinOnForeignKey
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function. This does
> > >>>>> make
> > >>>>>>>> for
> > >>>>>>>>>>> larger
> > >>>>>>>>>>>>>>>>>>> footprint,
> > >>>>>>>>>>>>>>>>>>>>>>>> but it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             removes all
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> agency
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> for resolving
> > >>>>>>> out-of-order
> > >>>>>>>>>>> arrivals
> > >>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> handling
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             CombinedKeys from
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> user. I believe
> that
> > >>>>>>> this
> > >>>>>>>>> makes
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> function
> > >>>>>>>>>>>>>>>>>>>>> much
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to use.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Let me know if this
> > >>>>>>> helps
> > >>>>>>>>>> resolve
> > >>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>>> questions,
> > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please feel
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> free to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> add anything else
> on
> > >>>>>>> your
> > >>>>>>>>> mind.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thanks again,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> On Tue, Sep 4, 2018
> > >>>>> at
> > >>>>>>> 8:36
> > >>>>>>>>> PM,
> > >>>>>>>>>>>>>>>>> Matthias J.
> > >>>>>>>>>>>>>>>>>>>>> Sax <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>> matthias@confluent.io
> > >>>>>>>>> <mailto:
> > >>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I am just catching
> > >>>>> up
> > >>>>>>> on
> > >>>>>>>>> this
> > >>>>>>>>>>>>>> thread. I
> > >>>>>>>>>>>>>>>>>>> did
> > >>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>> read
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             everything so
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> far,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> but want to share
> > >>>>>>> couple
> > >>>>>>>> of
> > >>>>>>>>>>>> initial
> > >>>>>>>>>>>>>>>>>>> thoughts:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Headers: I think
> > >>>>> there
> > >>>>>>> is
> > >>>>>>>> a
> > >>>>>>>>>>>>>> fundamental
> > >>>>>>>>>>>>>>>>>>>>>>>> difference
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             between header
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> usage
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> in this KIP and
> > >>>>> KP-258.
> > >>>>>>>> For
> > >>>>>>>>>> 258,
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>> add
> > >>>>>>>>>>>>>>>>>>>>> headers
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are owned by Kafka
> > >>>>>>> Streams
> > >>>>>>>>> and
> > >>>>>>>>>>>> nobody
> > >>>>>>>>>>>>>>>>>>> else is
> > >>>>>>>>>>>>>>>>>>>>>>>> supposed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > into
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> them. In fact, no
> > >>>>> user
> > >>>>>>>>> header
> > >>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>> written
> > >>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> thus, there are
> not
> > >>>>>>>>> conflicts.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Nevertheless, I
> > >>>>> don't
> > >>>>>>> see
> > >>>>>>>> a
> > >>>>>>>>>> big
> > >>>>>>>>>>>> issue
> > >>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headers within
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Streams.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> As long as we
> > >>>>> document
> > >>>>>>> it,
> > >>>>>>>>> we
> > >>>>>>>>>>> can
> > >>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>> "reserved"
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             header keys
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> users are not
> > >>>>> allowed
> > >>>>>>> to
> > >>>>>>>> use
> > >>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>> processing
> > >>>>>>>>>>>>>>>>>>>>>>>> data with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Kafka
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Streams.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this should
> be
> > >>>>>>> ok.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I think there is a
> > >>>>> safe
> > >>>>>>>> way
> > >>>>>>>>> to
> > >>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>>>>>> conflicts,
> > >>>>>>>>>>>>>>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > headers
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> only needed in
> > >>>>>>> internal
> > >>>>>>>>>> topics
> > >>>>>>>>>>> (I
> > >>>>>>>>>>>>>>>>> think):
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> For internal and
> > >>>>>>>> changelog
> > >>>>>>>>>>>> topics,
> > >>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>> namespace
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             all headers:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * user-defined
> > >>>>> headers
> > >>>>>>>> are
> > >>>>>>>>>>>>>> namespaced
> > >>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>>> "external."
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * internal
> headers
> > >>>>> are
> > >>>>>>>>>>>> namespaced as
> > >>>>>>>>>>>>>>>>>>>>>>>> "internal." +
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> While name
> spacing
> > >>>>>>> would
> > >>>>>>>> be
> > >>>>>>>>>>>>>> possible,
> > >>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> deserialize
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
> > >>>>>>> implies
> > >>>>>>>> a
> > >>>>>>>>>>>> runtime
> > >>>>>>>>>>>>>>>>>>> overhead.
> > >>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now
> to
> > >>>>>>> avoid
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>> overhead.
> > >>>>>>>>>>>>>>>>>>> If
> > >>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
> > >>>>>>> still
> > >>>>>>>> add
> > >>>>>>>>>>> name
> > >>>>>>>>>>>>>>>>> spacing
> > >>>>>>>>>>>>>>>>>>>>> later
> > >>>>>>>>>>>>>>>>>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> My main concern
> > >>>>> about
> > >>>>>>> the
> > >>>>>>>>>> design
> > >>>>>>>>>>>> it
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> type
> > >>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> If I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> understood the
> > >>>>> proposal
> > >>>>>>>>>>> correctly,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V1>
> > >>>>> table1 =
> > >>>>>>> ...
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K2,V2>
> > >>>>> table2 =
> > >>>>>>> ...
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V3>
> > >>>>>>> joinedTable
> > >>>>>>>> =
> > >>>>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> implies that the
> > >>>>>>>>> `joinedTable`
> > >>>>>>>>>>> has
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>>>>> as the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             left input
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this does
> not
> > >>>>>>> work
> > >>>>>>>>>> because
> > >>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>> table2
> > >>>>>>>>>>>>>>>>>>>>>>>> contains
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             multiple rows
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join with a record
> > >>>>> in
> > >>>>>>>> table1
> > >>>>>>>>>>>> (what is
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> main
> > >>>>>>>>>>>>>>>>>>>>>>>> purpose
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > foreign
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join), the result
> > >>>>> table
> > >>>>>>>>> would
> > >>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>> contain a
> > >>>>>>>>>>>>>>>>>>>>>>>> single
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             join result,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> multiple.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Example:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table1 input
> stream:
> > >>>>>>> <A,X>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table2 input
> stream:
> > >>>>>>>>>> <a,(A,1)>,
> > >>>>>>>>>>>>>>>>> <b,(A,2)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> We use table2
> value
> > >>>>> a
> > >>>>>>>>> foreign
> > >>>>>>>>>>> key
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> table1
> > >>>>>>>>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             "A" joins).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result key is the
> > >>>>> same
> > >>>>>>> key
> > >>>>>>>>> as
> > >>>>>>>>>>> key
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>> table1,
> > >>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             implies that the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result can either
> be
> > >>>>>>> <A,
> > >>>>>>>>>>>> join(X,1)>
> > >>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>> <A,
> > >>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             but not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > both.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Because the share
> > >>>>> the
> > >>>>>>> same
> > >>>>>>>>>> key,
> > >>>>>>>>>>>>>>>>> whatever
> > >>>>>>>>>>>>>>>>>>>>> result
> > >>>>>>>>>>>>>>>>>>>>>>>> record
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we emit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > later,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> overwrite the
> > >>>>> previous
> > >>>>>>>>> result.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This is the reason
> > >>>>> why
> > >>>>>>> Jan
> > >>>>>>>>>>>> originally
> > >>>>>>>>>>>>>>>>>>> proposed
> > >>>>>>>>>>>>>>>>>>>>>>>> to use
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > combination
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> both primary keys
> of
> > >>>>>>> the
> > >>>>>>>>> input
> > >>>>>>>>>>>> tables
> > >>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             output table.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> makes the keys of
> > >>>>> the
> > >>>>>>>> output
> > >>>>>>>>>>> table
> > >>>>>>>>>>>>>>>>> unique
> > >>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             store both in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> output table:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Result would be
> > >>>>> <A-a,
> > >>>>>>>>>>> join(X,1)>,
> > >>>>>>>>>>>>>> <A-b,
> > >>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> On 9/4/18 1:36 PM,
> > >>>>> Jan
> > >>>>>>>>>> Filipiak
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Just on remark
> here.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> The
> high-watermark
> > >>>>>>> could
> > >>>>>>>> be
> > >>>>>>>>>>>>>>>>> disregarded.
> > >>>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>>>>>> decision
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             about the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> forward
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> depends on the
> > >>>>> size of
> > >>>>>>>> the
> > >>>>>>>>>>>>>> aggregated
> > >>>>>>>>>>>>>>>>>>> map.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> Only 1 element
> long
> > >>>>>>> maps
> > >>>>>>>>>> would
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>> unpacked
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             forwarded. 0
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > element
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> maps
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> would be
> published
> > >>>>> as
> > >>>>>>>>> delete.
> > >>>>>>>>>>> Any
> > >>>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>> count
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of map entries is
> > >>>>> in
> > >>>>>>>>> "waiting
> > >>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>> correct
> > >>>>>>>>>>>>>>>>>>>>>>>> deletes to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > arrive"-state.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> On 04.09.2018
> > >>>>> 21:29,
> > >>>>>>> Adam
> > >>>>>>>>>>>> Bellemare
> > >>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> It does look
> like I
> > >>>>>>> could
> > >>>>>>>>>>> replace
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> second
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartition store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> highwater store
> > >>>>> with
> > >>>>>>> a
> > >>>>>>>>>> groupBy
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>> reduce.
> > >>>>>>>>>>>>>>>>>>>>>>>> However,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             it looks
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> still need to
> > >>>>> store
> > >>>>>>> the
> > >>>>>>>>>>>> highwater
> > >>>>>>>>>>>>>>>>> value
> > >>>>>>>>>>>>>>>>>>>>> within
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             materialized
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> store,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> compare the
> > >>>>> arrival of
> > >>>>>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>>>>>> (assuming
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> understanding
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> THIS is
> > >>>>> correct...).
> > >>>>>>> This
> > >>>>>>>>> in
> > >>>>>>>>>>>> effect
> > >>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> now,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> just with the two
> > >>>>>>> tables
> > >>>>>>>>>> merged
> > >>>>>>>>>>>>>>>>> together.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> --
> > >>>>>>>> -- Guozhang
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>> --
> > >>>>> -- Guozhang
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Jan

Ahh, I got it! It is deterministic once you apply the groupBy function you
mentioned a few months ago to the output, but not before you apply it...
correct? I was not thinking about the groupBy function.

Here's how I understand how it could work from an API perspective: I am
going to use the terminology "KScatteredTable" to represent the
intermediate table that is not yet resolved - basically the join was
performed but no race condition handling is done.

If I wanted to join three KTables together on foreign keys, one of the ways
I could do it is:

KScatteredTable scatteredOne =  ktableOne.oneToManyJoin(kTableTwo,
joinerFuncTwo, foreignKeyExtractorTwo);
KScatteredTable scatteredTwo = scatteredOne.oneToManyJoin(kTableThree,
joinerFuncThree, foreignKeyExtractorThree)

//Now I groupBy the key that I want to obtain, and I can resolve the out of
order dependencies here.
scatteredTwo.groupBy( keyValueMapper )   ( shown here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-SolutionB-User-ManagedGroupBy(Jan's)
)

Is this in line with what you're doing? Can this be done without exposing
the CombinedKey? As you mentioned before "A Table
KTable<CombinedKey<A,B>,JoinedResult> is not a good return type. It breaks
the KTable invariant that a table is currently partitioned by its key".
With that being said, are the only two operations that a KScatteredTable
would need to support be oneToManyJoin and groupBy?

Thanks for your thoughts

Adam


On Wed, Jan 2, 2019 at 3:07 PM Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi Adam,
>
> I am kinda surprised! Yes my solution of course is correct. Don't really
> know what to show in an example as I am convinced you grabbed the
> concept of how mine works,
>
> If there is a race condition there is a race condition. It doesn't
> matter if there is 10 minutes or milliseconds between events. Either
> they are properly guarded or not. My solution has no such race
> condition. It is 'eventual consistent'. You gonna see all sort of stuff
> coming up during a reprocess.
>
> The user can still fk it up later though. But that is usual business.
>
> In reality I try to supress updates from left sides as long as possible
> because right side updates are more expensive if left is already
> fullish. So that limits the space a little but there are no grantees.
> The result however, after lag is zero is the same every time.
>
> The trade-offs can be shifted as you like. My solution gives full power
> to the user and only does a minimum in the framework. You push
> everything into streams.
>
> If you ask me, not a good choice. Will anyone listen. No.
> I do actually think its to late to do my way. It's not like if you
> haven't been gone through the effort and building it.
>
> Just wanted to give you guys another chance, to think it through  ;)
>
> Regarding what will be observed. I consider it a plus that all events
> that are in the inputs have an respective output. Whereas your solution
> might "swallow" events.
>
> Best Jan
>
>
> On 02.01.2019 15:30, Adam Bellemare wrote:
> > Jan
> >
> > I have been thinking a lot about the history of the discussion and your
> > original proposal, and why you believe it is a better solution. The
> biggest
> > problem with your original proposed design is that it seems to me to be
> > non-deterministic. It is subject to race conditions that are dependent
> > entirely on the data, and without resolution of these races you can end
> up
> > with different results each time. If I am mistaken and this is indeed
> > deterministic, then please let me know and provide an explanation,
> ideally
> > with an example.
> >
> > The way I see it is that you will get very different answers to your
> > non-race-condition-resolved join topology, especially if you are nesting
> it
> > with additional joins as you have indicated you are doing. Consider
> > rebuilding an application state from the beginning of two topics. If the
> > left/this side has multiple foreign-key changes in a row, spaced out
> every
> > ten minutes, you may see something like this:
> >
> > (foo, foreignKey=red) t=0
> > (foo, foreignKey=blue) t=0+10m
> > (foo, foreignKey=green) t=0+20m
> > (foo, foreignKey=purple) t=0+30m
> > (foo, foreignKey=blue) t=0+40m
> > (foo, foreignKey=white) t=0+50m
> >
> > During realtime processing, all of the updates may have correctly
> > propagated because it took less than 10 minutes to resolve each join.
> Upon
> > rebuilding from the start, however, all of these events would be
> processed
> > in quick succession. The presence or absence of data will affect the
> > results of your join, and the results can vary with each run depending on
> > the data. Because of this, I cannot support any kind of solution that
> would
> > allow the exposure of an unresolved intermediate state. I can understand
> if
> > you don't support this, but this is why, as you said, you have the
> freedom
> > to use the Processor API.
> >
> >
> > With that being said, either the solution that I originally proposed
> > (join's ocurring on the foreign node) or John + Guozhang's solution
> > (registering with the foreign node for notifications) is fine with me -
> > both have the same API and we can evaluate it further during
> implementation.
> >
> >
> > Thanks
> >
> > Adam
> >
> > On Thu, Dec 27, 2018 at 2:38 PM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >> Hi,
> >>
> >> just want to let you guys know that this thing is spiralling out of
> >> control if you ask me.
> >>
> >> First you take away the possibility for the user to optimize. Now you
> >> pile up complexity to perform some afterwards optimisation, that from my
> >> POV completely misses the point. As if the actual call to the joiner
> >> really gonna be an expensive part. It wont. Truth is, you don't have a
> >> clue which side is gonna be smaller. might be the key you shuffle around
> >> is >>> than the value on the other side already.
> >>
> >> You know my opinion on this. For me its dead, I just leave you the
> >> message here as an opportunity to reconsider the choices that were made.
> >>
> >> Whish y'll a happy new year :)
> >>
> >>
> >>
> >>
> >>
> >>
> >> On 27.12.2018 17:22, Adam Bellemare wrote:
> >>> Hi All
> >>>
> >>> Sorry for the delay - holidays and all. I have since updated the KIP
> with
> >>> John's original suggestion and have pruned a number of the no longer
> >>> relevant diagrams. Any more comments would be welcomed, otherwise I
> will
> >>> look to kick off the vote again shortly.
> >>>
> >>> Thanks
> >>> Adam
> >>>
> >>> On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <
> adam.bellemare@gmail.com
> >>>
> >>> wrote:
> >>>
> >>>> Hi John and Guozhang
> >>>>
> >>>> Ah yes, I lost that in the mix! Thanks for the convergent solutions -
> I
> >> do
> >>>> think that the attachment that John included makes for a better
> design.
> >> It
> >>>> should also help with overall performance as very high-cardinality
> >> foreign
> >>>> keyed data (say millions of events with the same entity) will be able
> to
> >>>> leverage the multiple nodes for join functionality instead of having
> it
> >> all
> >>>> performed in one node. There is still a bottleneck in the right table
> >>>> having to propagate all those events, but with slimmer structures,
> less
> >> IO
> >>>> and no need to perform the join I think the throughput will be much
> >> higher
> >>>> in those scenarios.
> >>>>
> >>>> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
> >>>> version of John's diagram and ensure that the example flow matches
> >>>> correctly. Then I can go back to working on the PR to match the
> diagram.
> >>>>
> >>>> Thanks both of you for all the help - very much appreciated.
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hi John,
> >>>>>
> >>>>> Just made a pass on your diagram (nice hand-drawing btw!), and
> >> obviously
> >>>>> we
> >>>>> are thinking about the same thing :) A neat difference that I like,
> is
> >>>>> that
> >>>>> in the pre-join repartition topic we can still send message in the
> >> format
> >>>>> of `K=k, V=(i=2)` while using "i" as the partition key in
> >>>>> StreamsPartition,
> >>>>> this way we do not need to even augment the key for the repartition
> >> topic,
> >>>>> but just do a projection on the foreign key part but trim all other
> >>>>> fields:
> >>>>> as long as we still materialize the store as `A-2` co-located with
> the
> >>>>> right KTable, that is fine.
> >>>>>
> >>>>> As I mentioned in my previous email, I also think this has a few
> >>>>> advantages
> >>>>> on saving over-the-wire bytes as well as disk bytes.
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io>
> >> wrote:
> >>>>>
> >>>>>> Hi Guozhang,
> >>>>>>
> >>>>>> Thanks for taking a look! I think Adam's already addressed your
> >>>>> questions
> >>>>>> as well as I could have.
> >>>>>>
> >>>>>> Hi Adam,
> >>>>>>
> >>>>>> Thanks for updating the KIP. It looks great, especially how all the
> >>>>>> need-to-know information is right at the top, followed by the
> details.
> >>>>>>
> >>>>>> Also, thanks for that high-level diagram. Actually, now that I'm
> >> looking
> >>>>>> at it, I think part of my proposal got lost in translation,
> although I
> >>>>> do
> >>>>>> think that what you have there is also correct.
> >>>>>>
> >>>>>> I sketched up a crude diagram based on yours and attached it to the
> >> KIP
> >>>>>> (I'm not sure if attached or inline images work on the mailing
> list):
> >>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
> >>>>>> . It's also attached to this email for convenience.
> >>>>>>
> >>>>>> Hopefully, you can see how it's intended to line up, and which parts
> >> are
> >>>>>> modified.
> >>>>>> At a high level, instead of performing the join on the right-hand
> >> side,
> >>>>>> we're essentially just registering interest, like "LHS key A wishes
> to
> >>>>>> receive updates for RHS key 2". Then, when there is a new "interest"
> >> or
> >>>>> any
> >>>>>> updates to the RHS records, it "broadcasts" its state back to the
> LHS
> >>>>>> records who are interested in it.
> >>>>>>
> >>>>>> Thus, instead of sending the LHS values to the RHS joiner workers
> and
> >>>>> then
> >>>>>> sending the join results back to the LHS worke be co-partitioned and
> >>>>>> validated, we instead only send the LHS *keys* to the RHS workers
> and
> >>>>> then
> >>>>>> only the RHS k/v back to be joined by the LHS worker.
> >>>>>>
> >>>>>> I've been considering both your diagram and mine, and I *think* what
> >> I'm
> >>>>>> proposing has a few advantages.
> >>>>>>
> >>>>>> Here are some points of interest as you look at the diagram:
> >>>>>> * When we extract the foreign key and send it to the Pre-Join
> >>>>> Repartition
> >>>>>> Topic, we can send only the FK/PK pair. There's no need to worry
> about
> >>>>>> custom partitioner logic, since we can just use the foreign key
> >> plainly
> >>>>> as
> >>>>>> the repartition record key. Also, we save on transmitting the LHS
> >> value,
> >>>>>> since we only send its key in this step.
> >>>>>> * We also only need to store the RHSKey:LHSKey mapping in the
> >>>>>> MaterializedSubscriptionStore, saving on disk. We can use the same
> >> rocks
> >>>>>> key format you proposed and the same algorithm involving range scans
> >>>>> when
> >>>>>> the RHS records get updated.
> >>>>>> * Instead of joining on the right side, all we do is compose a
> >>>>>> re-repartition record so we can broadcast the RHS k/v pair back to
> the
> >>>>>> original LHS partition. (this is what the "rekey" node is doing)
> >>>>>> * Then, there is a special kind of Joiner that's co-resident in the
> >> same
> >>>>>> StreamTask as the LHS table, subscribed to the Post-Join Repartition
> >>>>> Topic.
> >>>>>> ** This Joiner is *not* triggered directly by any changes in the LHS
> >>>>>> KTable. Instead, LHS events indirectly trigger the join via the
> whole
> >>>>>> lifecycle.
> >>>>>> ** For each event arriving from the Post-Join Repartition Topic, the
> >>>>>> Joiner looks up the corresponding record in the LHS KTable. It
> >> validates
> >>>>>> the FK as you noted, discarding any inconsistent events. Otherwise,
> it
> >>>>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the
> >> join
> >>>>>> result
> >>>>>> ** Note that the Joiner itself is stateless, so materializing the
> join
> >>>>>> result is optional, just as with the 1:1 joins.
> >>>>>>
> >>>>>> So in summary:
> >>>>>> * instead of transmitting the LHS keys and values to the right and
> the
> >>>>>> JoinResult back to the left, we only transmit the LHS keys to the
> >> right
> >>>>> and
> >>>>>> the RHS values to the left. Assuming the average RHS value is on
> >> smaller
> >>>>>> than or equal to the average join result size, it's a clear win on
> >>>>> broker
> >>>>>> traffic. I think this is actually a reasonable assumption, which we
> >> can
> >>>>>> discuss more if you're suspicious.
> >>>>>> * we only need one copy of the data (the left and right tables need
> to
> >>>>> be
> >>>>>> materialized) and one extra copy of the PK:FK pairs in the
> >> Materialized
> >>>>>> Subscription Store. Materializing the join result is optional, just
> as
> >>>>> with
> >>>>>> the existing 1:1 joins.
> >>>>>> * we still need the fancy range-scan algorithm on the right to
> locate
> >>>>> all
> >>>>>> interested LHS keys when a RHS value is updated, but we don't need a
> >>>>> custom
> >>>>>> partitioner for either repartition topic (this is of course a
> >>>>> modification
> >>>>>> we could make to your version as well)
> >>>>>>
> >>>>>> How does this sound to you? (And did I miss anything?)
> >>>>>> -John
> >>>>>>
> >>>>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
> >>>>> adam.bellemare@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hi John & Guozhang
> >>>>>>>
> >>>>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up
> the
> >>>>> KIP,
> >>>>>>> pruned much of what I wrote and put a simplified diagram near the
> top
> >>>>> to
> >>>>>>> illustrate the workflow. I encapsulated Jan's content at the bottom
> >> of
> >>>>> the
> >>>>>>> document. I believe it is simpler to read by far now.
> >>>>>>>
> >>>>>>> @Guozhang Wang <wa...@gmail.com>:
> >>>>>>>> #1: rekey left table
> >>>>>>>>     -> source from the left upstream, send to rekey-processor to
> >>>>> generate
> >>>>>>> combined key, and then sink to copartition topic.
> >>>>>>> Correct.
> >>>>>>>
> >>>>>>>> #2: first-join with right table
> >>>>>>>>     -> source from the right table upstream, materialize the right
> >>>>> table.
> >>>>>>>>     -> source from the co-partition topic, materialize the rekeyed
> >> left
> >>>>>>> table, join with the right table, rekey back, and then sink to the
> >>>>>>> rekeyed-back topic.
> >>>>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I need
> >> the
> >>>>>>> Foreign-Key value generated in #1 above to compare in the
> resolution
> >>>>>>> stage.
> >>>>>>>
> >>>>>>>> #3: second join
> >>>>>>>>      -> source from the rekeyed-back topic, materialize the
> rekeyed
> >>>>> back
> >>>>>>> table.
> >>>>>>>>     -> source from the left upstream, materialize the left table,
> >> join
> >>>>>>> with
> >>>>>>> the rekeyed back table.
> >>>>>>> Almost - As each event comes in, we just run it through a stateful
> >>>>>>> processor that checks the original ("This") KTable for the key. The
> >>>>> value
> >>>>>>> payload then has the foreignKeyExtractor applied again as in Part
> #1
> >>>>>>> above,
> >>>>>>> and gets the current foreign key. Then we compare it to the joined
> >>>>> event
> >>>>>>> that we are currently resolving. If they have the same foreign-key,
> >>>>>>> propagate the result out. If they don't, throw the event away.
> >>>>>>>
> >>>>>>> The end result is that we do need to materialize 2 additional
> tables
> >>>>>>> (left/this-combinedkey table, and the final Joined table) as I've
> >>>>>>> illustrated in the updated KIP. I hope the diagram clears it up a
> lot
> >>>>>>> better. Please let me know.
> >>>>>>>
> >>>>>>> Thanks again
> >>>>>>> Adam
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>>>
> >>>>>>>> John,
> >>>>>>>>
> >>>>>>>> Thanks a lot for the suggestions on refactoring the wiki, I agree
> >>>>> with
> >>>>>>> you
> >>>>>>>> that we should consider the KIP proposal to be easily understood
> by
> >>>>>>> anyone
> >>>>>>>> in the future to read, and hence should provide a good summary on
> >> the
> >>>>>>>> user-facing interfaces, as well as rejected alternatives to
> >> represent
> >>>>>>>> briefly "how we came a long way to this conclusion, and what we
> have
> >>>>>>>> argued, disagreed, and agreed about, etc" so that readers do not
> >>>>> need to
> >>>>>>>> dig into the DISCUSS thread to get all the details. We can, of
> >>>>> course,
> >>>>>>> keep
> >>>>>>>> the implementation details like "workflows" on the wiki page as a
> >>>>>>> addendum
> >>>>>>>> section since it also has correlations.
> >>>>>>>>
> >>>>>>>> Regarding your proposal on comment 6): that's a very interesting
> >>>>> idea!
> >>>>>>> Just
> >>>>>>>> to clarify that I understands it fully correctly: the proposal's
> >>>>>>> resulted
> >>>>>>>> topology is still the same as the current proposal, where we will
> >>>>> have 3
> >>>>>>>> sub-topologies for this operator:
> >>>>>>>>
> >>>>>>>> #1: rekey left table
> >>>>>>>>      -> source from the left upstream, send to rekey-processor to
> >>>>> generate
> >>>>>>>> combined key, and then sink to copartition topic.
> >>>>>>>>
> >>>>>>>> #2: first-join with right table
> >>>>>>>>      -> source from the right table upstream, materialize the
> right
> >>>>> table.
> >>>>>>>>      -> source from the co-partition topic, materialize the
> rekeyed
> >>>>> left
> >>>>>>>> table, join with the right table, rekey back, and then sink to the
> >>>>>>>> rekeyed-back topic.
> >>>>>>>>
> >>>>>>>> #3: second join
> >>>>>>>>      -> source from the rekeyed-back topic, materialize the
> rekeyed
> >>>>> back
> >>>>>>>> table.
> >>>>>>>>      -> source from the left upstream, materialize the left table,
> >> join
> >>>>>>> with
> >>>>>>>> the rekeyed back table.
> >>>>>>>>
> >>>>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology
> since
> >>>>>>> both of
> >>>>>>>> them read from the left table source stream. In this workflow, we
> >>>>> need
> >>>>>>> to
> >>>>>>>> materialize 4 tables (left table in #3, right table in #2, rekeyed
> >>>>> left
> >>>>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
> >>>>>>>> (copartition topic, rekeyed-back topic).
> >>>>>>>>
> >>>>>>>> Compared with Adam's current proposal in the workflow overview, it
> >>>>> has
> >>>>>>> the
> >>>>>>>> same num.materialize tables (left table, rekeyed left table, right
> >>>>>>> table,
> >>>>>>>> out-of-ordering resolver table), and same num.internal topics
> (two).
> >>>>> The
> >>>>>>>> advantage is that on the copartition topic, we can save bandwidth
> by
> >>>>> not
> >>>>>>>> sending value, and in #2 the rekeyed left table is smaller since
> we
> >>>>> do
> >>>>>>> not
> >>>>>>>> have any values to materialize. Is that right?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
> >>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Adam,
> >>>>>>>>>
> >>>>>>>>> Given that the committers are all pretty busy right now, I think
> >>>>> that
> >>>>>>> it
> >>>>>>>>> would help if you were to refactor the KIP a little to reduce the
> >>>>>>>> workload
> >>>>>>>>> for reviewers.
> >>>>>>>>>
> >>>>>>>>> I'd recommend the following changes:
> >>>>>>>>> * relocate all internal details to a section at the end called
> >>>>>>> something
> >>>>>>>>> like "Implementation Notes" or something like that.
> >>>>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
> >>>>> mention
> >>>>>>>> only
> >>>>>>>>> publicly-facing API changes.
> >>>>>>>>> ** for example, the interface that you've already listed there,
> as
> >>>>>>> well
> >>>>>>>> as
> >>>>>>>>> a textual description of the guarantees we'll be providing (join
> >>>>>>> result
> >>>>>>>> is
> >>>>>>>>> copartitioned with the LHS, and the join result is guaranteed
> >>>>> correct)
> >>>>>>>>>
> >>>>>>>>> A good target would be that the whole main body of the KIP,
> >>>>> including
> >>>>>>>>> Status, Motivation, Proposal, Justification, and Rejected
> >>>>> Alternatives
> >>>>>>>> all
> >>>>>>>>> fit "above the fold" (i.e., all fit on the screen at a
> comfortable
> >>>>>>> zoom
> >>>>>>>>> level).
> >>>>>>>>> I think the only real Rejected Alternative that bears mention at
> >>>>> this
> >>>>>>>> point
> >>>>>>>>> is KScatteredTable, which you could just include the executive
> >>>>>>> summary on
> >>>>>>>>> (no implementation details), and link to extra details in the
> >>>>>>>>> Implementation Notes section.
> >>>>>>>>>
> >>>>>>>>> Taking a look at the wiki page, ~90% of the text there is
> internal
> >>>>>>>> detail,
> >>>>>>>>> which is useful for the dubious, but doesn't need to be ratified
> >>>>> in a
> >>>>>>>> vote
> >>>>>>>>> (and would be subject to change without notice in the future
> >>>>> anyway).
> >>>>>>>>> There's also a lot of conflicting discussion, as you've very
> >>>>>>> respectfully
> >>>>>>>>> tried to preserve the original proposal from Jan while adding
> your
> >>>>>>> own.
> >>>>>>>>> Isolating all this information in a dedicated section at the
> bottom
> >>>>>>> frees
> >>>>>>>>> the voters up to focus on the public API part of the proposal,
> >>>>> which
> >>>>>>> is
> >>>>>>>>> really all they need to consider.
> >>>>>>>>>
> >>>>>>>>> Plus, it'll be clear to future readers which parts of the
> document
> >>>>> are
> >>>>>>>>> enduring, and which parts are a snapshot of our implementation
> >>>>>>> thinking
> >>>>>>>> at
> >>>>>>>>> the time.
> >>>>>>>>>
> >>>>>>>>> I'm suggesting this because I suspect that the others haven't
> made
> >>>>>>> time
> >>>>>>>> to
> >>>>>>>>> review it partly because it seems daunting. If it seems like it
> >>>>> would
> >>>>>>> be
> >>>>>>>> a
> >>>>>>>>> huge time investment to review, people will just keep putting it
> >>>>> off.
> >>>>>>> But
> >>>>>>>>> if the KIP is a single page, then they'll be more inclined to
> give
> >>>>> it
> >>>>>>> a
> >>>>>>>>> read.
> >>>>>>>>>
> >>>>>>>>> Honestly, I don't think the KIP itself is that controversial
> (apart
> >>>>>>> from
> >>>>>>>>> the scattered table thing (sorry, Jan) ). Most of the discussion
> >>>>> has
> >>>>>>> been
> >>>>>>>>> around the implementation, which we can continue more effectively
> >>>>> in
> >>>>>>> a PR
> >>>>>>>>> once the KIP has passed.
> >>>>>>>>>
> >>>>>>>>> How does that sound?
> >>>>>>>>> -John
> >>>>>>>>>
> >>>>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
> >>>>>>> adam.bellemare@gmail.com
> >>>>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> 1) I believe that the resolution mechanism John has proposed is
> >>>>>>>>> sufficient
> >>>>>>>>>> - it is clean and easy and doesn't require additional RocksDB
> >>>>>>> stores,
> >>>>>>>>> which
> >>>>>>>>>> reduces the footprint greatly. I don't think we need to resolve
> >>>>>>> based
> >>>>>>>> on
> >>>>>>>>>> timestamp or offset anymore, but if we decide to do to that
> >>>>> would be
> >>>>>>>>> within
> >>>>>>>>>> the bounds of the existing API.
> >>>>>>>>>>
> >>>>>>>>>> 2) Is the current API sufficient, or does it need to be altered
> >>>>> to
> >>>>>>> go
> >>>>>>>>> back
> >>>>>>>>>> to vote?
> >>>>>>>>>>
> >>>>>>>>>> 3) KScatteredTable implementation can always be added in a
> future
> >>>>>>>>> revision.
> >>>>>>>>>> This API does not rule it out. This implementation of this
> >>>>> function
> >>>>>>>> would
> >>>>>>>>>> simply be replaced with `KScatteredTable.resolve()` while still
> >>>>>>>>> maintaining
> >>>>>>>>>> the existing API, thereby giving both features as Jan outlined
> >>>>>>> earlier.
> >>>>>>>>>> Would this work?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks Guozhang, John and Jan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <
> john@confluent.io
> >>>>>>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi, all,
> >>>>>>>>>>>
> >>>>>>>>>>>>> In fact, we
> >>>>>>>>>>>>> can just keep a single final-result store with timestamps
> >>>>> and
> >>>>>>>> reject
> >>>>>>>>>>> values
> >>>>>>>>>>>>> that have a smaller timestamp, is that right?
> >>>>>>>>>>>
> >>>>>>>>>>>> Which is the correct output should at least be decided on the
> >>>>>>>> offset
> >>>>>>>>> of
> >>>>>>>>>>>> the original message.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for this point, Jan.
> >>>>>>>>>>>
> >>>>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
> >>>>> the
> >>>>>>> k/v
> >>>>>>>>>>> store,
> >>>>>>>>>>> as well as providing a storage-format upgrade path.
> >>>>>>>>>>>
> >>>>>>>>>>> I might have missed it, but I think we have yet to discuss
> >>>>> whether
> >>>>>>>> it's
> >>>>>>>>>>> safe
> >>>>>>>>>>> or desirable just to swap topic-ordering our for
> >>>>>>> timestamp-ordering.
> >>>>>>>>> This
> >>>>>>>>>>> is
> >>>>>>>>>>> a very deep topic, and I think it would only pollute the
> >>>>> current
> >>>>>>>>>>> discussion.
> >>>>>>>>>>>
> >>>>>>>>>>> What Adam has proposed is safe, given the *current* ordering
> >>>>>>>> semantics
> >>>>>>>>>>> of the system. If we can agree on his proposal, I think we can
> >>>>>>> merge
> >>>>>>>>> the
> >>>>>>>>>>> feature well before the conversation about timestamp ordering
> >>>>> even
> >>>>>>>>> takes
> >>>>>>>>>>> place, much less reaches a conclusion. In the mean time, it
> >>>>> would
> >>>>>>>> seem
> >>>>>>>>> to
> >>>>>>>>>>> be unfortunate to have one join operator with different
> >>>>> ordering
> >>>>>>>>>> semantics
> >>>>>>>>>>> from every other KTable operator.
> >>>>>>>>>>>
> >>>>>>>>>>> If and when that timestamp discussion takes place, many (all?)
> >>>>>>> KTable
> >>>>>>>>>>> operations
> >>>>>>>>>>> will need to be updated, rendering the many:one join a small
> >>>>>>> marginal
> >>>>>>>>>> cost.
> >>>>>>>>>>>
> >>>>>>>>>>> And, just to plug it again, I proposed an algorithm above that
> >>>>> I
> >>>>>>>>> believe
> >>>>>>>>>>> provides
> >>>>>>>>>>> correct ordering without any additional metadata, and
> >>>>> regardless
> >>>>>>> of
> >>>>>>>> the
> >>>>>>>>>>> ordering semantics. I didn't bring it up further, because I
> >>>>> felt
> >>>>>>> the
> >>>>>>>>> KIP
> >>>>>>>>>>> only needs
> >>>>>>>>>>> to agree on the public API, and we can discuss the
> >>>>> implementation
> >>>>>>> at
> >>>>>>>>>>> leisure in
> >>>>>>>>>>> a PR...
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> -John
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> >>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
> >>>>>>>>>>>>> Hello Adam / Jan / John,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Sorry for being late on this thread! I've finally got some
> >>>>>>> time
> >>>>>>>>> this
> >>>>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
> >>>>> I've
> >>>>>>>> also
> >>>>>>>>>>>> realized
> >>>>>>>>>>>>> there are a bunch of other things I need to enqueue while
> >>>>>>>> cleaning
> >>>>>>>>>> them
> >>>>>>>>>>>> up
> >>>>>>>>>>>>> --- sth I need to improve on my side). So here are my
> >>>>>>> thoughts:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regarding the APIs: I like the current written API in the
> >>>>> KIP.
> >>>>>>>> More
> >>>>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
> >>>>>>>>> functionalities
> >>>>>>>>>> as
> >>>>>>>>>>>>> well as 2) other join types than inner as separate KIPs
> >>>>> since
> >>>>>>> 1)
> >>>>>>>>> may
> >>>>>>>>>>>> worth
> >>>>>>>>>>>>> a general API refactoring that can benefit not only
> >>>>> foreignkey
> >>>>>>>>> joins
> >>>>>>>>>>> but
> >>>>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> >>>>>>>>>>>> ),
> >>>>>>>>>>>>> and I'm not sure if other join types would actually be
> >>>>> needed
> >>>>>>>>> (maybe
> >>>>>>>>>>> left
> >>>>>>>>>>>>> join still makes sense), so it's better to
> >>>>>>>>>>> wait-for-people-to-ask-and-add
> >>>>>>>>>>>>> than add-sth-that-no-one-uses.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
> >>>>> a
> >>>>>>>>>>>>> KScatteredTable for users to inject their own optimization:
> >>>>>>> I'd
> >>>>>>>>>> prefer
> >>>>>>>>>>> to
> >>>>>>>>>>>>> do the current option as-is, and my main rationale is for
> >>>>>>>>>> optimization
> >>>>>>>>>>>>> rooms inside the Streams internals and the API
> >>>>> succinctness.
> >>>>>>> For
> >>>>>>>>>>> advanced
> >>>>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
> >>>>> own
> >>>>>>>>>>>> optimization,
> >>>>>>>>>>>>> while it is too much of the work to use Processor API
> >>>>>>> directly, I
> >>>>>>>>>> think
> >>>>>>>>>>>> we
> >>>>>>>>>>>>> can still extend the current API to support it in the
> >>>>> future
> >>>>>>> if
> >>>>>>>> it
> >>>>>>>>>>>> becomes
> >>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>
> >>>>>>>>>>>> no internal optimization potential. it's a myth
> >>>>>>>>>>>>
> >>>>>>>>>>>> ¯\_(ツ)_/¯
> >>>>>>>>>>>>
> >>>>>>>>>>>> :-)
> >>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
> >>>>> as
> >>>>>>> I
> >>>>>>>>>>> mentioned
> >>>>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
> >>>>> key-value
> >>>>>>>>> store)
> >>>>>>>>>>> we
> >>>>>>>>>>>>> can actually make this step simpler than the current
> >>>>>>> proposal. In
> >>>>>>>>>> fact,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>> can just keep a single final-result store with timestamps
> >>>>> and
> >>>>>>>>> reject
> >>>>>>>>>>>> values
> >>>>>>>>>>>>> that have a smaller timestamp, is that right?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Which is the correct output should at least be decided on the
> >>>>>>>> offset
> >>>>>>>>> of
> >>>>>>>>>>>> the original message.
> >>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
> >>>>>>> Adam
> >>>>>>>> to
> >>>>>>>>>>> make
> >>>>>>>>>>>>> such HUGE progress on this KIP!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> >>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> If they don't find the time:
> >>>>>>>>>>>>>> They usually take the opposite path from me :D
> >>>>>>>>>>>>>> so the answer would be clear.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> hence my suggestion to vote.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
> >>>>>>>>>>>>>>> Hi Guozhang and Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
> >>>>> KIP
> >>>>>>>> to a
> >>>>>>>>>>> point
> >>>>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
> >>>>> a
> >>>>>>>>>>> tie-breaker,
> >>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>> you will). If you have anyone else you may think should
> >>>>>>> look at
> >>>>>>>>>> this,
> >>>>>>>>>>>>>>> please tag them accordingly.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The scenario is as such:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Current Option:
> >>>>>>>>>>>>>>> API:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> >>>>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
> >>>>>>>>> partition
> >>>>>>>>>>> with
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> foreignKey (repartition 1)
> >>>>>>>>>>>>>>> 2) Join the data
> >>>>>>>>>>>>>>> 3) Shuffle the data back to the original node
> >>>>> (repartition
> >>>>>>> 2)
> >>>>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
> >>>>>>>>> foreign-key
> >>>>>>>>>>>>>> changes.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Alternate Option:
> >>>>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
> >>>>>>>>>>>>>>> - It would be keyed on a wrapped key function:
> >>>>>>> <CombinedKey<KO,
> >>>>>>>>> K>,
> >>>>>>>>>>> VR>
> >>>>>>>>>>>>>> (KO
> >>>>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
> >>>>> Result)
> >>>>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
> >>>>>>>>> otherwise a
> >>>>>>>>>>>> user
> >>>>>>>>>>>>>>> would be able to perform additional functions directly
> >>>>> from
> >>>>>>> the
> >>>>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
> >>>>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
> >>>>>>> tradeoffs.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
> >>>>> possible,
> >>>>>>>> but
> >>>>>>>>>> will
> >>>>>>>>>>>>>>> require for implementation details to be made in the API
> >>>>> and
> >>>>>>>> some
> >>>>>>>>>>>>>> exposure
> >>>>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I appreciate any insight into this.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> >>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi John
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
> >>>>>>> summary
> >>>>>>>> is
> >>>>>>>>>>>>>> accurate
> >>>>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
> >>>>> that
> >>>>>>>>> there
> >>>>>>>>>>> is a
> >>>>>>>>>>>>>> risk
> >>>>>>>>>>>>>>>> of inconsistent final states without performing the
> >>>>>>>> resolution.
> >>>>>>>>>> This
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> major concern for me as most of the data I have dealt
> >>>>> with
> >>>>>>> is
> >>>>>>>>>>> produced
> >>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>> relational databases. We have seen a number of cases
> >>>>> where
> >>>>>>> a
> >>>>>>>>> user
> >>>>>>>>>> in
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
> >>>>>>> they
> >>>>>>>>> made
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>> mistake, and then updated the field again with a new
> >>>>> key.
> >>>>>>> The
> >>>>>>>>>> events
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>>> propagated out as they are produced, and as such we have
> >>>>>>> had
> >>>>>>>>>>>> real-world
> >>>>>>>>>>>>>>>> cases where these inconsistencies were propagated
> >>>>>>> downstream
> >>>>>>>> as
> >>>>>>>>>> the
> >>>>>>>>>>>>>> final
> >>>>>>>>>>>>>>>> values due to the race conditions in the fanout of the
> >>>>>>> data.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> This solution that I propose values correctness of the
> >>>>>>> final
> >>>>>>>>>> result
> >>>>>>>>>>>> over
> >>>>>>>>>>>>>>>> other factors.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> We could always move this function over to using a
> >>>>>>>>> KScatteredTable
> >>>>>>>>>>>>>>>> implementation in the future, and simply deprecate it
> >>>>> this
> >>>>>>>> join
> >>>>>>>>>> API
> >>>>>>>>>>> in
> >>>>>>>>>>>>>>>> time. I think I would like to hear more from some of the
> >>>>>>> other
> >>>>>>>>>> major
> >>>>>>>>>>>>>>>> committers on which course of action they would think is
> >>>>>>> best
> >>>>>>>>>> before
> >>>>>>>>>>>> any
> >>>>>>>>>>>>>>>> more coding is done.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks again
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> >>>>>>>> john@confluent.io>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Jan and Adam,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
> >>>>> are
> >>>>>>>>>>> encouraging.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
> >>>>>>> agree
> >>>>>>>>> that
> >>>>>>>>>>>>>> avoiding
> >>>>>>>>>>>>>>>>> unnecessary join outputs is especially important when
> >>>>> the
> >>>>>>>>> fan-out
> >>>>>>>>>>> is
> >>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>> high. I suppose this could also be built into the
> >>>>>>>>> implementation
> >>>>>>>>>>>> we're
> >>>>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
> >>>>>>> KIP
> >>>>>>>>>> (since
> >>>>>>>>>>>>>> it's an
> >>>>>>>>>>>>>>>>> API-transparent optimization).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
> >>>>>>> didn't
> >>>>>>>>>> bring
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>> up
> >>>>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
> >>>>> the
> >>>>>>>> KIP
> >>>>>>>>>>> as-is,
> >>>>>>>>>>>>>>>>> despite the disagreement.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
> >>>>>>>>>> reasonable.
> >>>>>>>>>>>>>>>>> It sounds like Jan values more the potential for
> >>>>>>> developers
> >>>>>>>> to
> >>>>>>>>>>>> optimize
> >>>>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
> >>>>> whereas
> >>>>>>>> Adam
> >>>>>>>>>>>> places
> >>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>> value on having a single operator that people can use
> >>>>>>> without
> >>>>>>>>>> extra
> >>>>>>>>>>>>>> steps
> >>>>>>>>>>>>>>>>> at the end.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Personally, although I do find it exceptionally
> >>>>> annoying
> >>>>>>>> when a
> >>>>>>>>>>>>>> framework
> >>>>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
> >>>>> it
> >>>>>>>> seems
> >>>>>>>>>>> better
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>>>> for a single operation.
> >>>>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
> >>>>>>> significant
> >>>>>>>>>>>> latitude
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> the implementation (for example, joining only at the
> >>>>> end,
> >>>>>>> not
> >>>>>>>>> in
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> middle
> >>>>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
> >>>>> resolution;
> >>>>>>> how
> >>>>>>>> we
> >>>>>>>>>>>>>> represent
> >>>>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
> >>>>>>> vectors),
> >>>>>>>>>>> etc.).
> >>>>>>>>>>>>>> If we
> >>>>>>>>>>>>>>>>> publish something like a KScatteredTable with the
> >>>>>>>>>> right-partitioned
> >>>>>>>>>>>>>> joined
> >>>>>>>>>>>>>>>>> data, then the API pretty much locks in the
> >>>>>>> implementation as
> >>>>>>>>>> well.
> >>>>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
> >>>>> mean
> >>>>>>>>> "seems";
> >>>>>>>>>>> if
> >>>>>>>>>>>>>>>>> anyone
> >>>>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
> >>>>>>>>> simpler,
> >>>>>>>>>> I
> >>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
> >>>>>>> algebra
> >>>>>>>>>>>>>> perspective,
> >>>>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
> >>>>>>> KTable
> >>>>>>>>> in
> >>>>>>>>>>> all
> >>>>>>>>>>>>>>>>> cases.
> >>>>>>>>>>>>>>>>> * That said, there might still be room in the API for a
> >>>>>>>>> different
> >>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
> >>>>> then
> >>>>>>> do
> >>>>>>>>>> things
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
> >>>>> haven't
> >>>>>>>>> thought
> >>>>>>>>>>>>>> through
> >>>>>>>>>>>>>>>>> all the consequences yet.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> This is all just my opinion after thinking over the
> >>>>>>>> discussion
> >>>>>>>>> so
> >>>>>>>>>>>>>> far...
> >>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> >>>>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
> >>>>> the
> >>>>>>>>>>>> prefixScan.
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> have attached the file, but I will also include the
> >>>>> text
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>> body
> >>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
> >>>>>>>> attached
> >>>>>>>>>>>> files).
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
> >>>>> large
> >>>>>>>>> values
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
> >>>>>>> would
> >>>>>>>> be
> >>>>>>>>>> more
> >>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
> >>>>>>> For
> >>>>>>>>>>> instance,
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
> >>>>> and a
> >>>>>>>>> further
> >>>>>>>>>>>>>>>>> ~35-85mS
> >>>>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
> >>>>>>> them.
> >>>>>>>>>>> Iterating
> >>>>>>>>>>>>>>>>>> through the records just to generate a simple count
> >>>>>>> takes ~
> >>>>>>>> 40
> >>>>>>>>>>> times
> >>>>>>>>>>>>>>>>> longer
> >>>>>>>>>>>>>>>>>> than the flush + scan combined.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>>>>>>>>>>>>>> Setup:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
> >>>>>>>>> (Xmx512m,
> >>>>>>>>>>>>>> Xms512m)
> >>>>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
> >>>>> directly-accessible
> >>>>>>>>> Kafka
> >>>>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
> >>>>> the
> >>>>>>>>>>>>>>>>>> ProcessorContext).
> >>>>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
> >>>>> values
> >>>>>>>>>> provided
> >>>>>>>>>>> in
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> 2.1/trunk release.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> keysize = 128 bytes
> >>>>>>>>>>>>>>>>>> valsize = 512 bytes
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Step 1:
> >>>>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
> >>>>>>>> left-padded
> >>>>>>>>>>>>>>>>>> auto-incrementing integer)
> >>>>>>>>>>>>>>>>>> Step 2:
> >>>>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
> >>>>>>>>>>>>>> auto-incrementing
> >>>>>>>>>>>>>>>>>> integer)
> >>>>>>>>>>>>>>>>>> Step 3:
> >>>>>>>>>>>>>>>>>> Perform flush
> >>>>>>>>>>>>>>>>>> Step 4:
> >>>>>>>>>>>>>>>>>> Perform prefixScan
> >>>>>>>>>>>>>>>>>> Step 5:
> >>>>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
> >>>>> count of
> >>>>>>>>>> expected
> >>>>>>>>>>>>>>>>> events.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>>>>>>>>>>>>>> Results:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>>>>>>>>>>>>>> X = 1k (11k events total)
> >>>>>>>>>>>>>>>>>> Flush Time = 39 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 7 mS
> >>>>>>>>>>>>>>>>>> 6.9 MB disk
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>>> X = 10k (110k events total)
> >>>>>>>>>>>>>>>>>> Flush Time = 45 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 8 mS
> >>>>>>>>>>>>>>>>>> 127 MB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>>> X = 100k (1.1M events total)
> >>>>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>>>> Flush Time = 60 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 12 mS
> >>>>>>>>>>>>>>>>>> 678 MB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>>>> Flush Time = 45 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 7 mS
> >>>>>>>>>>>>>>>>>> 576 MB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>>> X = 1MB (11M events total)
> >>>>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>>>> Flush Time = 52 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 19 mS
> >>>>>>>>>>>>>>>>>> 7.2 GB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>>>> Flush Time = 84 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 34 mS
> >>>>>>>>>>>>>>>>>> 9.1 GB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
> >>>>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>>>> Flush Time = 82 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 63 mS
> >>>>>>>>>>>>>>>>>> 17GB - 276 sst files
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>>>> Flush Time = 116 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 35 mS
> >>>>>>>>>>>>>>>>>> 23GB - 361 sst files
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Test3:
> >>>>>>>>>>>>>>>>>> Flush Time = 103 mS
> >>>>>>>>>>>>>>>>>> Scan Time = 82 mS
> >>>>>>>>>>>>>>>>>> 19 GB - 300 sst files
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
> >>>>>>> events. I
> >>>>>>>>>> tried
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
> >>>>> the
> >>>>>>>> 100GB+
> >>>>>>>>>>> range
> >>>>>>>>>>>>>>>>> and my
> >>>>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
> >>>>> be
> >>>>>>> done
> >>>>>>>>>> but I
> >>>>>>>>>>>>>>>>> suspect
> >>>>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
> >>>>>>>> results
> >>>>>>>>>>>> above.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> At this point in time, I think the only major
> >>>>> discussion
> >>>>>>>> point
> >>>>>>>>>> is
> >>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
> >>>>> repartitioning
> >>>>>>>> back +
> >>>>>>>>>>>>>> resolving
> >>>>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
> >>>>> the
> >>>>>>>> client
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks folks,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> >>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
> >>>>> the
> >>>>>>> 2.1
> >>>>>>>>>>> release
> >>>>>>>>>>>>>>>>>>> caused an
> >>>>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
> >>>>>>> radar
> >>>>>>>>>> (which
> >>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
> >>>>>>> some
> >>>>>>>>>> extend
> >>>>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
> >>>>>>> keep) me
> >>>>>>>>>>>>>>>>> preoccupied.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
> >>>>> guess
> >>>>>>> Jan
> >>>>>>>>> was
> >>>>>>>>>>> on
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>> track!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
> >>>>> and
> >>>>>>> the
> >>>>>>>>>> prior
> >>>>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> API Thoughts:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
> >>>>>>>> many-to-one
> >>>>>>>>>>> join.
> >>>>>>>>>>>>>>>>> Could
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
> >>>>>>> flip
> >>>>>>>>> the
> >>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
> >>>>> join
> >>>>>>>>> type,
> >>>>>>>>>>> and
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
> >>>>>>>>> one-to-one
> >>>>>>>>>>>> join.
> >>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>> would work, of course, but it would be super
> >>>>>>> inefficient
> >>>>>>>>>>> compared
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> simple rekey-and-join.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
> >>>>>>>> specified
> >>>>>>>>>>>> whether
> >>>>>>>>>>>>>>>>>>> it's an
> >>>>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
> >>>>>>> join, as
> >>>>>>>>>>>>>>>>> (neglecting
> >>>>>>>>>>>>>>>>>>> IQ),
> >>>>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
> >>>>>>> it in
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> ValueJoiner.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
> >>>>> quite
> >>>>>>>>> right.
> >>>>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
> >>>>>>>>> paradigms
> >>>>>>>>>> in
> >>>>>>>>>>>>>>>>> play in
> >>>>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
> >>>>>>> three
> >>>>>>>>>>>> Serialized
> >>>>>>>>>>>>>>>>>>> args, I
> >>>>>>>>>>>>>>>>>>>> think it would be better to have one that allows
> >>>>>>>>> (optionally)
> >>>>>>>>>>>>>> setting
> >>>>>>>>>>>>>>>>>>> the 4
> >>>>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
> >>>>>>>>>>> Materialized.
> >>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
> >>>>>>> already
> >>>>>>>>> be
> >>>>>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
> >>>>>>> config
> >>>>>>>>>> might
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> applicable.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
> >>>>> joins
> >>>>>>>>> don't
> >>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>>> setting
> >>>>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
> >>>>>>> since
> >>>>>>>>> the
> >>>>>>>>>>>> rekey
> >>>>>>>>>>>>>>>>>>>> operation needs to produce results that are
> >>>>>>> co-partitioned
> >>>>>>>>>> with
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> "other"
> >>>>>>>>>>>>>>>>>>>> KTable.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
> >>>>>>> didn't
> >>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
> >>>>>>>>>> "deserializing"
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
> >>>>>>> think
> >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
> >>>>>>> namespace
> >>>>>>>> at
> >>>>>>>>>>> source
> >>>>>>>>>>>>>>>>> nodes
> >>>>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
> >>>>>>> practically
> >>>>>>>> no
> >>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
> >>>>> API
> >>>>>>>> change
> >>>>>>>>>> wrt
> >>>>>>>>>>>>>>>>> headers
> >>>>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
> >>>>> placed
> >>>>>>> on
> >>>>>>>>>> users'
> >>>>>>>>>>>>>>>>>>> headers.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
> >>>>> the
> >>>>>>>>> header
> >>>>>>>>>> at
> >>>>>>>>>>>>>>>>> all...
> >>>>>>>>>>>>>>>>>>>> stay tuned)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
> >>>>>>> table
> >>>>>>>>>>> growing
> >>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
> >>>>>>>>> implementing
> >>>>>>>>>>> OCC
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
> >>>>> the
> >>>>>>>>> rekey
> >>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
> >>>>> changes.
> >>>>>>> As
> >>>>>>>>> such,
> >>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
> >>>>> offset in
> >>>>>>>> the
> >>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
> >>>>> the
> >>>>>>> same
> >>>>>>>>>>> number
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> keys
> >>>>>>>>>>>>>>>>>>>> as the source table at all times.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
> >>>>>>> might
> >>>>>>>> be
> >>>>>>>>>>>> relevant
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
> >>>>>>> timestamp
> >>>>>>>>> in
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
> >>>>> the
> >>>>>>>>>> timestamp
> >>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> OCC
> >>>>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
> >>>>>>> want to
> >>>>>>>>> use
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
> >>>>> use a
> >>>>>>>>> similar
> >>>>>>>>>>>>>>>>>>> mechanism to
> >>>>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
> >>>>>>>> re-keyed
> >>>>>>>>>>>> values,
> >>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>> you could avoid a separate table.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
> >>>>> long
> >>>>>>>>> time,
> >>>>>>>>>> so
> >>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
> >>>>> if we
> >>>>>>>> can
> >>>>>>>>>>> avoid
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> HWM
> >>>>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
> >>>>> final
> >>>>>>>> join
> >>>>>>>>>>>>>>>>> instead...
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
> >>>>> Letter
> >>>>>>>> FK,
> >>>>>>>>>>> (other
> >>>>>>>>>>>>>>>>>>> data))
> >>>>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Left table:
> >>>>>>>>>>>>>>>>>>>> 1: (A, xyz)
> >>>>>>>>>>>>>>>>>>>> 2: (B, asd)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Right table:
> >>>>>>>>>>>>>>>>>>>> A: EntityA
> >>>>>>>>>>>>>>>>>>>> B: EntityB
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
> >>>>>>> key,
> >>>>>>>> but
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> propagating the value at all..
> >>>>>>>>>>>>>>>>>>>> Rekey table:
> >>>>>>>>>>>>>>>>>>>> A-1: (dummy value)
> >>>>>>>>>>>>>>>>>>>> B-2: (dummy value)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
> >>>>>>>>>>>>>>>>>>>> A-1: EntityA
> >>>>>>>>>>>>>>>>>>>> B-2: EntityB
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Which gets rekeyed back:
> >>>>>>>>>>>>>>>>>>>> 1: A, EntityA
> >>>>>>>>>>>>>>>>>>>> 2: B, EntityB
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> And finally we do the actual join:
> >>>>>>>>>>>>>>>>>>>> Result table:
> >>>>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
> >>>>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
> >>>>>>>> opportunity
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> compare
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
> >>>>>>> the
> >>>>>>>>> right
> >>>>>>>>>>>>>>>>> table. If
> >>>>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
> >>>>>>> must be
> >>>>>>>>>>>> outdated.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
> >>>>> xyz)
> >>>>>>> gets
> >>>>>>>>>>> updated
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> (1:
> >>>>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
> >>>>> whether
> >>>>>>> the
> >>>>>>>>>> final
> >>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
> >>>>>>>> algorithm
> >>>>>>>>>>> above,
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
> >>>>>>> (B,
> >>>>>>>>> xyz),
> >>>>>>>>>>> (B,
> >>>>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
> >>>>>>>>> information
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
> >>>>>>> mentioned
> >>>>>>>>>> often
> >>>>>>>>>>> I
> >>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
> >>>>>>>>> automatically.
> >>>>>>>>>> I
> >>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>> this is the real performance eater ;)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
> >>>>>>>>>> performance
> >>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> range scans when records change in the right table.
> >>>>>>> You've
> >>>>>>>>>> said
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> you've
> >>>>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
> >>>>>>> for a
> >>>>>>>>>>> while.
> >>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
> >>>>>>> you've
> >>>>>>>>>>>> observed?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
> >>>>>>> topmost
> >>>>>>>>>> thing
> >>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
> >>>>>>>>> performance
> >>>>>>>>>>> of
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
> >>>>>>> 500k -
> >>>>>>>>> 1kk
> >>>>>>>>>>> rows
> >>>>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
> >>>>> the
> >>>>>>> work
> >>>>>>>>> you
> >>>>>>>>>>>> gotta
> >>>>>>>>>>>>>>>>>>> do, also when you pack your data into different
> >>>>> formats,
> >>>>>>>>>> usually
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
> >>>>> data
> >>>>>>> and
> >>>>>>>>> we
> >>>>>>>>>>>> can't
> >>>>>>>>>>>>>>>>>>> really change that. It is more important for users to
> >>>>>>>> prevent
> >>>>>>>>>>>> useless
> >>>>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
> >>>>> to
> >>>>>>> drop
> >>>>>>>>>>> changes
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> are not going to change my join output.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> usually it's:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
> >>>>>>>> old.equals(new)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
> >>>>> for
> >>>>>>>>>> smaller
> >>>>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
> >>>>> then
> >>>>>>>>> anyways.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
> >>>>>>> sure if
> >>>>>>>>>> it's
> >>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
> >>>>> the
> >>>>>>>>>> original
> >>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>> as I
> >>>>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
> >>>>> keys in
> >>>>>>>> the
> >>>>>>>>>>> value:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Left table:
> >>>>>>>>>>>>>>>>>>>> 1: A,...
> >>>>>>>>>>>>>>>>>>>> 2: B,...
> >>>>>>>>>>>>>>>>>>>> 3: A,...
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Gets re-keyed:
> >>>>>>>>>>>>>>>>>>>> A: [1, 3]
> >>>>>>>>>>>>>>>>>>>> B: [2]
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
> >>>>>>> regular
> >>>>>>>>>>>> single-key
> >>>>>>>>>>>>>>>>>>> lookup.
> >>>>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
> >>>>>>>> values,
> >>>>>>>>> as
> >>>>>>>>>>>>>>>>> there's
> >>>>>>>>>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
> >>>>> reference
> >>>>>>> rhs
> >>>>>>>>>>> records.
> >>>>>>>>>>>>>>>>>>> Offhand,
> >>>>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
> >>>>>>> past
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> threshold, we
> >>>>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
> >>>>> cases,
> >>>>>>> it
> >>>>>>>>> would
> >>>>>>>>>>> be
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
> >>>>> be
> >>>>>>> one
> >>>>>>>>> per
> >>>>>>>>>>>> (max
> >>>>>>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
> >>>>>>> there's
> >>>>>>>>> some
> >>>>>>>>>>>> extra
> >>>>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
> >>>>>>>> retraction.
> >>>>>>>>>> For
> >>>>>>>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
> >>>>> just
> >>>>>>> have
> >>>>>>>>> (A:
> >>>>>>>>>>>> [3]).
> >>>>>>>>>>>>>>>>>>> Some
> >>>>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
> >>>>>>> for 1
> >>>>>>>>> can
> >>>>>>>>>>> also
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> retracted.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> That's all!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
> >>>>> thoughtful
> >>>>>>>> KIP.
> >>>>>>>>>>> Sorry
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> discussion has been slow.
> >>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> >>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
> >>>>> up,
> >>>>>>> it
> >>>>>>>>> just
> >>>>>>>>>>>> goes
> >>>>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>>>> to discuss.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> would not expect to much attention with another
> >>>>>>> another
> >>>>>>>>> email
> >>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> best Jan
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>> Hello Contributors
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
> >>>>>>> need
> >>>>>>>> to
> >>>>>>>>>> bump
> >>>>>>>>>>>>>>>>> this to
> >>>>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
> >>>>>>> through
> >>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>> contributor
> >>>>>>>>>>>>>>>>>>>>>> feedback is given.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Main points that need addressing:
> >>>>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
> >>>>>>>>> singular
> >>>>>>>>>>>> graph
> >>>>>>>>>>>>>>>>> node
> >>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
> >>>>> right
> >>>>>>>>> now. I
> >>>>>>>>>>> am
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>> fuzzy
> >>>>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
> >>>>> I
> >>>>>>>> would
> >>>>>>>>>>>>>>>>> appreciate
> >>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>> help on this aspect.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
> >>>>>>> thread
> >>>>>>>>> has
> >>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>> discourse
> >>>>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
> >>>>> mark
> >>>>>>>>>> proposal
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> groupBy
> >>>>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
> >>>>> need
> >>>>>>> to
> >>>>>>>>>>> strictly
> >>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
> >>>>> up
> >>>>>>> to
> >>>>>>>> the
> >>>>>>>>>>>>>>>>> consumer.
> >>>>>>>>>>>>>>>>>>> Any
> >>>>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
> >>>>> this
> >>>>>>>> from
> >>>>>>>>>>>> moving
> >>>>>>>>>>>>>>>>> to a
> >>>>>>>>>>>>>>>>>>>>> vote?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >>>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
> >>>>>>>>>>>>>>>>> Stores.persistentWindowStore,
> >>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
> >>>>> segments
> >>>>>>>> you
> >>>>>>>>>> want
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>> large
> >>>>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
> >>>>>>> happens
> >>>>>>>>> is
> >>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
> >>>>> data
> >>>>>>> with
> >>>>>>>>> new
> >>>>>>>>>>>>>>>>>>> timestamps
> >>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
> >>>>> of
> >>>>>>> the
> >>>>>>>>>> work
> >>>>>>>>>>>> done
> >>>>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
> >>>>> the
> >>>>>>>>>> hopping
> >>>>>>>>>>>>>>>>> windowed
> >>>>>>>>>>>>>>>>>>>>> store.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
> >>>>>>> to be
> >>>>>>>>> two
> >>>>>>>>>>> 24h
> >>>>>>>>>>>>>>>>>>>>> segments.
> >>>>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
> >>>>> will
> >>>>>>> be
> >>>>>>>>>>> inserted
> >>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>> T in
> >>>>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
> >>>>> overlap
> >>>>>>> by
> >>>>>>>>> 12h.
> >>>>>>>>>>> As
> >>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>> goes on
> >>>>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
> >>>>> the
> >>>>>>>>> oldest
> >>>>>>>>>>>>>> segment
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
> >>>>> The
> >>>>>>>>>> records
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
> >>>>> it
> >>>>>>> is
> >>>>>>>>> the
> >>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>> time,
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
> >>>>> are
> >>>>>>>>>> retained
> >>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
> >>>>> TTL
> >>>>>>>> at a
> >>>>>>>>>>>>>>>>>>> segment-level,
> >>>>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
> >>>>>>>>>>>> ClassCastException
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>> similar.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
> >>>>>>>>> fetch/put
> >>>>>>>>>> or
> >>>>>>>>>>>>>>>>> iterate
> >>>>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
> >>>>>>> stream-time
> >>>>>>>>>> under
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> hoods
> >>>>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
> >>>>> all
> >>>>>>> the
> >>>>>>>>>>> windows
> >>>>>>>>>>>>>>>>>>> earlier?
> >>>>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
> >>>>>>> time
> >>>>>>>>>>>> gurantees.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Best
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
> >>>>>>> changed
> >>>>>>>>> the
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
> >>>>>>>> something
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>>> will be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> extremely
> >>>>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
> >>>>> said, I
> >>>>>>>> will
> >>>>>>>>>> not
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> moving
> >>>>>>>>>>>>>>>>>>>>>>>> ahead
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
> >>>>> reshuffle/groupBy
> >>>>>>>>> solution
> >>>>>>>>>>> as
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>> propose.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
> >>>>>>>> yourself
> >>>>>>>>>> off
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>> current PR
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
> >>>>>>> would
> >>>>>>>>> be
> >>>>>>>>>>> more
> >>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>> happy to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> >>>>>>>> stands
> >>>>>>>>>>> right
> >>>>>>>>>>>>>>>>> now,
> >>>>>>>>>>>>>>>>>>> I do
> >>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
> >>>>>>> alternatives
> >>>>>>>>>> without
> >>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
> >>>>> which
> >>>>>>>> they
> >>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> prefer.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
> >>>>> gave
> >>>>>>> up
> >>>>>>>> on
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>>>>> DSL
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
> >>>>> pulled
> >>>>>>>> this
> >>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>> through
> >>>>>>>>>>>>>>>>>>>>>>>> already.
> >>>>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
> >>>>> based on
> >>>>>>>>> PAPI.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
> >>>>>>> windowed
> >>>>>>>>>> state
> >>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
> >>>>> then I
> >>>>>>>> will
> >>>>>>>>>> come
> >>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
> >>>>> any of
> >>>>>>>> the
> >>>>>>>>>>>> binding
> >>>>>>>>>>>>>>>>>>> voters
> >>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
> >>>>>>> updated
> >>>>>>>> it
> >>>>>>>>>>>>>>>>> according
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
> >>>>> windowed
> >>>>>>>>> store.
> >>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
> >>>>> they
> >>>>>>>> are
> >>>>>>>>>>>>>>>>> completed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> >>>>>>>> already
> >>>>>>>>>>>> updated
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> PR?
> >>>>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
> >>>>> Missing
> >>>>>>>>>>> something?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
> >>>>> Wang <
> >>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
> >>>>> is
> >>>>>>> the
> >>>>>>>>>> wrong
> >>>>>>>>>>>>>> link,
> >>>>>>>>>>>>>>>>>>> as it
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
> >>>>>>> part of
> >>>>>>>>>>> KIP-258
> >>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
> >>>>>>>> KTable"
> >>>>>>>>>> such
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
> >>>>> materialized
> >>>>>>>> store,
> >>>>>>>>>>> i.e.
> >>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
> >>>>> older
> >>>>>>>> than
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>> key's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
> >>>>> ordering.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
> >>>>>>> Wang <
> >>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
> >>>>>>> final
> >>>>>>>>> step
> >>>>>>>>>>>> (i.e.
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> high
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
> >>>>>>> with
> >>>>>>>> a
> >>>>>>>>>>> window
> >>>>>>>>>>>>>>>>>>> store),
> >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
> >>>>>>> help:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
> >>>>>>> key-value
> >>>>>>>>>> store
> >>>>>>>>>>>>>>>>> (i.e.
> >>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
> >>>>>>> usage,
> >>>>>>>> as
> >>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
> >>>>>>>> ,
> >>>>>>>>> is
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
> >>>>> its
> >>>>>>>>>>> timestamp
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> smaller
> >>>>>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
> >>>>> I
> >>>>>>>> think
> >>>>>>>>> it
> >>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
> >>>>> based
> >>>>>>>>>>> filtering,
> >>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
> >>>>> the
> >>>>>>>>> joining
> >>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
> >>>>>>> final
> >>>>>>>>>> stage.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
> >>>>> and
> >>>>>>>> hence
> >>>>>>>>>>>>>>>>>>> non-windowed
> >>>>>>>>>>>>>>>>>>>>>>>> KTables
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
> >>>>>>> really
> >>>>>>>>> have
> >>>>>>>>>> a
> >>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>> support
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
> >>>>>>>>> KTable-KTable
> >>>>>>>>>>>>>>>>> non-key
> >>>>>>>>>>>>>>>>>>>>> joins
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
> >>>>> Filipiak
> >>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
> >>>>> would
> >>>>>>>> grow
> >>>>>>>>>>>>>> endlessly
> >>>>>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
> >>>>> pair
> >>>>>>> of
> >>>>>>>>>> (<this
> >>>>>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
> >>>>> is
> >>>>>>> used
> >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> differentiate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
> >>>>>>> proposal
> >>>>>>>>>> would
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> replace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
> >>>>>>> This
> >>>>>>>>> would
> >>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
> >>>>> time.
> >>>>>>> This
> >>>>>>>>>>> should
> >>>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> >>>>>>>> should
> >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> customizable
> >>>>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
> >>>>>>> perhaps
> >>>>>>>>> just
> >>>>>>>>>>> 10
> >>>>>>>>>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
> >>>>> do
> >>>>>>> the
> >>>>>>>>>> trick
> >>>>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>>>> Even
> >>>>>>>>>>>>>>>>>>>>>>>> if I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
> >>>>>>>>> repartitioning
> >>>>>>>>>>>>>>>>> optional
> >>>>>>>>>>>>>>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
> >>>>>>> am a
> >>>>>>>>>> little
> >>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>>> sceptical
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
> >>>>> one
> >>>>>>>>> could
> >>>>>>>>>>> run
> >>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
> >>>>>>> border. I
> >>>>>>>>> will
> >>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
> >>>>>>> problematic, we
> >>>>>>>>>> could
> >>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
> >>>>> performance
> >>>>>>>>> impact I
> >>>>>>>>>>>>>> guess.
> >>>>>>>>>>>>>>>>>>> Will
> >>>>>>>>>>>>>>>>>>>>>>>> let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
> >>>>> correct
> >>>>>>> as
> >>>>>>>>> is. I
> >>>>>>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> >>>>>>>>> timestamp(A)  <
> >>>>>>>>>>>>>>>>>>>>> timestamp(B).
> >>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
> >>>>> -
> >>>>>>>> thanks
> >>>>>>>>>> for
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> diagram, it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
> >>>>> do
> >>>>>>> not
> >>>>>>>>> have
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
> >>>>>>>>> available
> >>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>> would be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
> >>>>> Map.
> >>>>>>>> That
> >>>>>>>>>>> being
> >>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
> >>>>> for
> >>>>>>>>> clarity
> >>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> everyone
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
> >>>>> really
> >>>>>>> hard
> >>>>>>>>>> work.
> >>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> >>>>>>>>> original
> >>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
> >>>>> PAPI
> >>>>>>>> and
> >>>>>>>>>>>>>> basically
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
> >>>>> missed
> >>>>>>>> that
> >>>>>>>>> in
> >>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>> DSL
> >>>>>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
> >>>>>>> up on
> >>>>>>>>> my
> >>>>>>>>>>> end.
> >>>>>>>>>>>>>>>>> Will
> >>>>>>>>>>>>>>>>>>>>>>>> finish
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
> >>>>>>> week.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
> >>>>> the
> >>>>>>> Map
> >>>>>>>>> stay
> >>>>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> State
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
> >>>>> changes
> >>>>>>>> have
> >>>>>>>>>>>>>>>>> propagated?
> >>>>>>>>>>>>>>>>>>>>> Isn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
> >>>>>>> state
> >>>>>>>>>> store?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
> >>>>>>> substractor
> >>>>>>>> is
> >>>>>>>>>>> gonna
> >>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>> `null`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
> >>>>>>> there
> >>>>>>>> is
> >>>>>>>>>>> going
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
> >>>>> this
> >>>>>>>> store
> >>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
> >>>>> is a
> >>>>>>>>>> regular
> >>>>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>>> satisfying
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
> >>>>>>> join.
> >>>>>>>>> The
> >>>>>>>>>>>>>>>>> Windowed
> >>>>>>>>>>>>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
> >>>>> statefull
> >>>>>>>>>> operation
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
> >>>>>>> have
> >>>>>>>> the
> >>>>>>>>>>>> window
> >>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
> >>>>> custom
> >>>>>>>> group
> >>>>>>>>>> by
> >>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
> >>>>> key i
> >>>>>>>> think
> >>>>>>>>>> it
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> help
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
> >>>>> the
> >>>>>>>>>> original
> >>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
> >>>>>>> foundation
> >>>>>>>>> to
> >>>>>>>>>>>> build
> >>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
> >>>>> user.
> >>>>>>>> very
> >>>>>>>>>>>>>>>>>>> unfortunate. I
> >>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
> >>>>> do
> >>>>>>> not
> >>>>>>>>>> think
> >>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
> >>>>> Prajakta
> >>>>>>>>> Dumbre <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please remove me from this
> >>>>> group
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             On Tue, Sep 11, 2018 at 1:29 PM
> >>>>>>> Jan
> >>>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> >>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hi Adam,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > give me some time, will make
> >>>>>>> such a
> >>>>>>>>>>> chart.
> >>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>> time i
> >>>>>>>>>>>>>>>>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             get along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > well with giphy and ruined
> >>>>> all
> >>>>>>> your
> >>>>>>>>>>> charts.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hopefully i can get it done
> >>>>>>> today
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > On 08.09.2018 16:00, Adam
> >>>>>>> Bellemare
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Hi Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I have included a diagram
> >>>>> of
> >>>>>>>> what I
> >>>>>>>>>>>>>> attempted
> >>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I attempted this back at
> >>>>> the
> >>>>>>>> start
> >>>>>>>>> of
> >>>>>>>>>>> my
> >>>>>>>>>>>> own
> >>>>>>>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > solution, and since I could
> >>>>>>> not
> >>>>>>>> get
> >>>>>>>>>> it
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> work I
> >>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             discarded the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > code. At this point in
> >>>>> time,
> >>>>>>> if
> >>>>>>>> you
> >>>>>>>>>>> wish
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>>>> pursuing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             for your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > groupBy solution, I ask
> >>>>> that
> >>>>>>> you
> >>>>>>>>>> please
> >>>>>>>>>>>>>>>>> create a
> >>>>>>>>>>>>>>>>>>>>>>>> diagram on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > carefully explaining your
> >>>>>>>> solution.
> >>>>>>>>>>>> Please
> >>>>>>>>>>>>>>>>> feel
> >>>>>>>>>>>>>>>>>>> free
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the image I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > just posted as a starting
> >>>>>>> point.
> >>>>>>>> I
> >>>>>>>>> am
> >>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>>>> trouble
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             understanding your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > explanations but I think
> >>>>> that
> >>>>>>> a
> >>>>>>>>>>> carefully
> >>>>>>>>>>>>>>>>>>> constructed
> >>>>>>>>>>>>>>>>>>>>>>>> diagram
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             will clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > any misunderstandings.
> >>>>>>>> Alternately,
> >>>>>>>>>>>> please
> >>>>>>>>>>>>>>>>> post a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             comprehensive PR with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > your solution. I can only
> >>>>>>> guess
> >>>>>>>> at
> >>>>>>>>>> what
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> mean, and
> >>>>>>>>>>>>>>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             value my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > own
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > time as much as you value
> >>>>>>> yours,
> >>>>>>>> I
> >>>>>>>>>>>> believe
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             responsibility to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > provide an implementation
> >>>>>>> instead
> >>>>>>>>> of
> >>>>>>>>>> me
> >>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>>> guess.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > On Sat, Sep 8, 2018 at 8:00
> >>>>>>> AM,
> >>>>>>>> Jan
> >>>>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> >>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Hi James,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> nice to see you beeing
> >>>>>>>> interested.
> >>>>>>>>>>> kafka
> >>>>>>>>>>>>>>>>>>> streams at
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             point supports
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> all sorts of joins as
> >>>>> long as
> >>>>>>>> both
> >>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Adam is currently
> >>>>>>> implementing a
> >>>>>>>>>> join
> >>>>>>>>>>>>>> where a
> >>>>>>>>>>>>>>>>>>> KTable
> >>>>>>>>>>>>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             KTable can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> a one to many relation
> >>>>> ship
> >>>>>>>> (1:n).
> >>>>>>>>>> We
> >>>>>>>>>>>>>> exploit
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> rocksdb
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> datastore that keeps data
> >>>>>>> sorted
> >>>>>>>> (At
> >>>>>>>>>>> least
> >>>>>>>>>>>>>>>>>>> exposes an
> >>>>>>>>>>>>>>>>>>>>>>>> API to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> stored data in a sorted
> >>>>>>>> fashion).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I think the technical
> >>>>> caveats
> >>>>>>>> are
> >>>>>>>>>> well
> >>>>>>>>>>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > basically
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> down to philosophy and API
> >>>>>>>> Design
> >>>>>>>>> (
> >>>>>>>>>>> when
> >>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>> sees
> >>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>> newest
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             message).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I have a lengthy track
> >>>>>>> record of
> >>>>>>>>>>> loosing
> >>>>>>>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>>>> kinda
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arguments within
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> streams community and I
> >>>>> have
> >>>>>>> no
> >>>>>>>>> clue
> >>>>>>>>>>>> why.
> >>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> literally
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             can't wait for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> to churn through this
> >>>>> thread
> >>>>>>> and
> >>>>>>>>>> give
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> opinion on
> >>>>>>>>>>>>>>>>>>>>>>>> how we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > design
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the return type of the
> >>>>>>>>> oneToManyJoin
> >>>>>>>>>>> and
> >>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>>>>> power we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             want to give
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the user vs "simplicity"
> >>>>>>> (where
> >>>>>>>>>>>> simplicity
> >>>>>>>>>>>>>>>>> isn't
> >>>>>>>>>>>>>>>>>>>>>>>> really that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as users
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> need to understand it I
> >>>>>>> argue)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> waiting for you to join
> >>>>> in on
> >>>>>>>> the
> >>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Best Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> On 07.09.2018 15:49, James
> >>>>>>> Kwan
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I am new to this group
> >>>>> and I
> >>>>>>>>> found
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> subject
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             interesting.  Sounds
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> you guys want to
> >>>>> implement a
> >>>>>>>> join
> >>>>>>>>>>>> table of
> >>>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>>>>>>>> streams? Is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > somewhere
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I can see the original
> >>>>>>>>> requirement
> >>>>>>>>>> or
> >>>>>>>>>>>>>>>>> proposal?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> On Sep 7, 2018, at 8:13
> >>>>> AM,
> >>>>>>> Jan
> >>>>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
> >>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> On 05.09.2018 22:17,
> >>>>> Adam
> >>>>>>>>>> Bellemare
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> I'm currently testing
> >>>>>>> using a
> >>>>>>>>>>>> Windowed
> >>>>>>>>>>>>>>>>> Store
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> store the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             highwater
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> mark.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> By all indications this
> >>>>>>>> should
> >>>>>>>>>> work
> >>>>>>>>>>>>>> fine,
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> caveat
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             being that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> only resolve
> >>>>> out-of-order
> >>>>>>>>> arrival
> >>>>>>>>>>>> for up
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> size of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the window
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > (ie:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> 24h, 72h, etc). This
> >>>>> would
> >>>>>>>>> remove
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> possibility
> >>>>>>>>>>>>>>>>>>>>>>>> of it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > unbounded
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> size.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> With regards to Jan's
> >>>>>>>>>> suggestion, I
> >>>>>>>>>>>>>>>>> believe
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> remain in disagreement.
> >>>>>>>> While I
> >>>>>>>>>> do
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>> disagree
> >>>>>>>>>>>>>>>>>>>>>>>> with your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             statement
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> there likely to be
> >>>>>>> additional
> >>>>>>>>>> joins
> >>>>>>>>>>>> done
> >>>>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>>>>>>> real-world
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             workflow, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> see how you can
> >>>>>>> conclusively
> >>>>>>>>> deal
> >>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> foreign-key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> changes and subsequent
> >>>>>>>> joins. I
> >>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>> attempted
> >>>>>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             think you have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> proposed (without a
> >>>>>>>> high-water,
> >>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>> groupBy and
> >>>>>>>>>>>>>>>>>>>>>>>> reduce)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             and found
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> that if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> the foreign key changes
> >>>>>>> too
> >>>>>>>>>>> quickly,
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> load
> >>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             stream thread
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> too
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> high, the joined
> >>>>> messages
> >>>>>>>> will
> >>>>>>>>>>> arrive
> >>>>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>>>> and be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             incorrectly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> propagated, such that
> >>>>> an
> >>>>>>>>>>> intermediate
> >>>>>>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > final
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> event.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Can you shed some light
> >>>>> on
> >>>>>>>> your
> >>>>>>>>>>>> groupBy
> >>>>>>>>>>>>>>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             There must be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> some sort of flaw in it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I have a suspicion
> >>>>> where it
> >>>>>>>> is,
> >>>>>>>>> I
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             confirm. The idea
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> is bullet proof and it
> >>>>>>> must be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> an implementation mess
> >>>>> up.
> >>>>>>> I
> >>>>>>>>> would
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> clarify
> >>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we draw a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> conclusion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>    Repartitioning the
> >>>>>>>> scattered
> >>>>>>>>>>> events
> >>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> partitions is the only
> >>>>> way I
> >>>>>>>> know
> >>>>>>>>>> how
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> conclusively
> >>>>>>>>>>>>>>>>>>>>>>>> deal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> out-of-order events in
> >>>>> a
> >>>>>>>> given
> >>>>>>>>>> time
> >>>>>>>>>>>>>> frame,
> >>>>>>>>>>>>>>>>>>> and to
> >>>>>>>>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > data
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> eventually consistent
> >>>>> with
> >>>>>>>> the
> >>>>>>>>>>> input
> >>>>>>>>>>>>>>>>> events.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> If you have some code
> >>>>> to
> >>>>>>>> share
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> illustrates
> >>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             approach, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> very grateful as it
> >>>>> would
> >>>>>>>>> remove
> >>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>> misunderstandings
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that I may
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> ah okay you were looking
> >>>>>>> for
> >>>>>>>> my
> >>>>>>>>>>> code.
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             something easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> readable here as its
> >>>>>>> bloated
> >>>>>>>>> with
> >>>>>>>>>>>>>>>>> OO-patterns.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> its anyhow trivial:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> @Override
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      public T apply(K
> >>>>>>> aggKey,
> >>>>>>>> V
> >>>>>>>>>>>> value, T
> >>>>>>>>>>>>>>>>>>>>> aggregate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          Map<U, V>
> >>>>>>>>>>> currentStateAsMap =
> >>>>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             imaginary
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          U toModifyKey =
> >>>>>>>>>>>>>>>>> mapper.apply(value);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << this is
> >>>>> the
> >>>>>>>>> place
> >>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>> people
> >>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             gonna have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > issues
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and why you probably
> >>>>>>> couldn't
> >>>>>>>> do
> >>>>>>>>>> it.
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>> to find
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             a solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I didn't realize that
> >>>>> yet.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << we
> >>>>>>> propagate
> >>>>>>>> the
> >>>>>>>>>>>> field in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> joiner, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that we can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > pick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> it up in an aggregate.
> >>>>>>>> Probably
> >>>>>>>>>> you
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> thought
> >>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this in your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> approach right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I am
> >>>>> very
> >>>>>>> open
> >>>>>>>>> to
> >>>>>>>>>>>> find a
> >>>>>>>>>>>>>>>>>>> generic
> >>>>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             here. In my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> honest opinion this is
> >>>>>>> broken
> >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             looses
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the keys
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and only maintains the
> >>>>>>>> aggregate
> >>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I
> >>>>>>> abstracted
> >>>>>>>> it
> >>>>>>>>>> away
> >>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>> then way
> >>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> of oneToMany join. That
> >>>>> is
> >>>>>>>> why I
> >>>>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>>>> realize
> >>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             significance here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              <<
> >>>>> Opinions?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          for (V m :
> >>>>>>> current)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>> currentStateAsMap.put(mapper.apply(m),
> >>>>>>>>>>>>>> m);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          if (isAdder)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>> currentStateAsMap.put(toModifyKey,
> >>>>>>>>>>>>>> value);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          else
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>                  return
> >>>>>>> null;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          retrun
> >>>>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> On Wed, Sep 5, 2018 at
> >>>>>>> 3:35
> >>>>>>>> PM,
> >>>>>>>>>> Jan
> >>>>>>>>>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Jan.Filipiak@trivago.com
> >>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Thanks Adam for
> >>>>> bringing
> >>>>>>>>> Matthias
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> speed!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> about the
> >>>>> differences. I
> >>>>>>>> think
> >>>>>>>>>>>>>> re-keying
> >>>>>>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             optional at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> best.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I would say we return
> >>>>> a
> >>>>>>>>>>>> KScatteredTable
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> reshuffle()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>> KTable<originalKey,Joined>
> >>>>>>>> to
> >>>>>>>>>> make
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartitioning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> optional.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I am also in a big
> >>>>>>> favour of
> >>>>>>>>>> doing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             processing using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> group
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by instead high water
> >>>>>>> mark
> >>>>>>>>>>> tracking.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Just because unbounded
> >>>>>>>> growth
> >>>>>>>>> is
> >>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>> scary
> >>>>>>>>>>>>>>>>>>> + It
> >>>>>>>>>>>>>>>>>>>>>>>> saves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the header
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> stuff.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I think the
> >>>>> abstraction
> >>>>>>> of
> >>>>>>>>>> always
> >>>>>>>>>>>>>>>>>>> repartitioning
> >>>>>>>>>>>>>>>>>>>>>>>> back is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             just not so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> strong. Like the work
> >>>>> has
> >>>>>>>> been
> >>>>>>>>>>> done
> >>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             back and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> grouping
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by something else
> >>>>>>> afterwards
> >>>>>>>>> is
> >>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>> common.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> On 05.09.2018 13:49,
> >>>>> Adam
> >>>>>>>>>>> Bellemare
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Hi Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thank you for your
> >>>>>>>> feedback,
> >>>>>>>>> I
> >>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>> appreciate
> >>>>>>>>>>>>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> While name spacing
> >>>>>>> would be
> >>>>>>>>>>>> possible,
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > deserialize
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
> >>>>>>> implies
> >>>>>>>> a
> >>>>>>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now to
> >>>>>>> avoid
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
> >>>>>>> still
> >>>>>>>> add
> >>>>>>>>>>> name
> >>>>>>>>>>>>>>>>> spacing
> >>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Agreed. I will go
> >>>>> with
> >>>>>>>>> using a
> >>>>>>>>>>>>>> reserved
> >>>>>>>>>>>>>>>>>>> string
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             document it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> My main concern about
> >>>>>>> the
> >>>>>>>>>> design
> >>>>>>>>>>> it
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> type of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> understood the
> >>>>> proposal
> >>>>>>>>>>> correctly,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> In your example, you
> >>>>>>> have
> >>>>>>>>>> table1
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> table2
> >>>>>>>>>>>>>>>>>>>>>>>> swapped.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Here is how it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> works
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> currently:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 1) table1 has the
> >>>>>>> records
> >>>>>>>>> that
> >>>>>>>>>>>> contain
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> foreign key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             within their
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> value.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 input stream:
> >>>>>>>>>>>> <a,(fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table2 input stream:
> >>>>>>> <A,X>,
> >>>>>>>>>> <B,Y>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 2) A Value mapper is
> >>>>>>>> required
> >>>>>>>>>> to
> >>>>>>>>>>>>>> extract
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> foreign
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 foreign key
> >>>>>>> mapper:
> >>>>>>>> (
> >>>>>>>>>>> value
> >>>>>>>>>>>> =>
> >>>>>>>>>>>>>>>>>>> value.fk
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <http://value.fk> )
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> The mapper is
> >>>>> applied to
> >>>>>>>> each
> >>>>>>>>>>>> element
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> table1,
> >>>>>>>>>>>>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             new combined
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> key is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> made:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 mapped: <A-a,
> >>>>>>>>>>> (fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <B-c,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 3) The rekeyed events
> >>>>>>> are
> >>>>>>>>>>>>>> copartitioned
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> table2:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> a) Stream Thread with
> >>>>>>>>> Partition
> >>>>>>>>>>> 0:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> RepartitionedTable1:
> >>>>>>> <A-a,
> >>>>>>>>>>>>>>>>> (fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             (fk=A,bar=2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <A,X>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> b) Stream Thread with
> >>>>>>>>> Partition
> >>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> RepartitionedTable1:
> >>>>>>> <B-c,
> >>>>>>>>>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 4) From here, they
> >>>>> can
> >>>>>>> be
> >>>>>>>>>> joined
> >>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>> locally
> >>>>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             applying the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> joiner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> At this point, Jan's
> >>>>>>> design
> >>>>>>>>> and
> >>>>>>>>>>> my
> >>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>>>>> deviate. My
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design goes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> repartition the data
> >>>>>>>>> post-join
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> resolve
> >>>>>>>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arrival of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> records,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> finally returning the
> >>>>>>> data
> >>>>>>>>>> keyed
> >>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> original key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             I do not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> expose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> CombinedKey or any of
> >>>>>>> the
> >>>>>>>>>>> internals
> >>>>>>>>>>>>>>>>>>> outside of
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             joinOnForeignKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function. This does
> >>>>> make
> >>>>>>>> for
> >>>>>>>>>>> larger
> >>>>>>>>>>>>>>>>>>> footprint,
> >>>>>>>>>>>>>>>>>>>>>>>> but it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             removes all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> agency
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> for resolving
> >>>>>>> out-of-order
> >>>>>>>>>>> arrivals
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             CombinedKeys from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> user. I believe that
> >>>>>>> this
> >>>>>>>>> makes
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> function
> >>>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to use.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Let me know if this
> >>>>>>> helps
> >>>>>>>>>> resolve
> >>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>> questions,
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please feel
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> free to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> add anything else on
> >>>>>>> your
> >>>>>>>>> mind.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thanks again,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> On Tue, Sep 4, 2018
> >>>>> at
> >>>>>>> 8:36
> >>>>>>>>> PM,
> >>>>>>>>>>>>>>>>> Matthias J.
> >>>>>>>>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>> matthias@confluent.io
> >>>>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I am just catching
> >>>>> up
> >>>>>>> on
> >>>>>>>>> this
> >>>>>>>>>>>>>> thread. I
> >>>>>>>>>>>>>>>>>>> did
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> read
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             everything so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> far,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> but want to share
> >>>>>>> couple
> >>>>>>>> of
> >>>>>>>>>>>> initial
> >>>>>>>>>>>>>>>>>>> thoughts:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Headers: I think
> >>>>> there
> >>>>>>> is
> >>>>>>>> a
> >>>>>>>>>>>>>> fundamental
> >>>>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             between header
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> usage
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> in this KIP and
> >>>>> KP-258.
> >>>>>>>> For
> >>>>>>>>>> 258,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>> headers
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are owned by Kafka
> >>>>>>> Streams
> >>>>>>>>> and
> >>>>>>>>>>>> nobody
> >>>>>>>>>>>>>>>>>>> else is
> >>>>>>>>>>>>>>>>>>>>>>>> supposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> them. In fact, no
> >>>>> user
> >>>>>>>>> header
> >>>>>>>>>>> are
> >>>>>>>>>>>>>>>>> written
> >>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> thus, there are not
> >>>>>>>>> conflicts.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Nevertheless, I
> >>>>> don't
> >>>>>>> see
> >>>>>>>> a
> >>>>>>>>>> big
> >>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headers within
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Streams.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> As long as we
> >>>>> document
> >>>>>>> it,
> >>>>>>>>> we
> >>>>>>>>>>> can
> >>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>> "reserved"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             header keys
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> users are not
> >>>>> allowed
> >>>>>>> to
> >>>>>>>> use
> >>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>> data with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Streams.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this should be
> >>>>>>> ok.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I think there is a
> >>>>> safe
> >>>>>>>> way
> >>>>>>>>> to
> >>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>> conflicts,
> >>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > headers
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> only needed in
> >>>>>>> internal
> >>>>>>>>>> topics
> >>>>>>>>>>> (I
> >>>>>>>>>>>>>>>>> think):
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> For internal and
> >>>>>>>> changelog
> >>>>>>>>>>>> topics,
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>> namespace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             all headers:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * user-defined
> >>>>> headers
> >>>>>>>> are
> >>>>>>>>>>>>>> namespaced
> >>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>> "external."
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * internal headers
> >>>>> are
> >>>>>>>>>>>> namespaced as
> >>>>>>>>>>>>>>>>>>>>>>>> "internal." +
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> While name spacing
> >>>>>>> would
> >>>>>>>> be
> >>>>>>>>>>>>>> possible,
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> deserialize
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
> >>>>>>> implies
> >>>>>>>> a
> >>>>>>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now to
> >>>>>>> avoid
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
> >>>>>>> still
> >>>>>>>> add
> >>>>>>>>>>> name
> >>>>>>>>>>>>>>>>> spacing
> >>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> My main concern
> >>>>> about
> >>>>>>> the
> >>>>>>>>>> design
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> If I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> understood the
> >>>>> proposal
> >>>>>>>>>>> correctly,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V1>
> >>>>> table1 =
> >>>>>>> ...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K2,V2>
> >>>>> table2 =
> >>>>>>> ...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V3>
> >>>>>>> joinedTable
> >>>>>>>> =
> >>>>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> implies that the
> >>>>>>>>> `joinedTable`
> >>>>>>>>>>> has
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>> as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             left input
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this does not
> >>>>>>> work
> >>>>>>>>>> because
> >>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>> table2
> >>>>>>>>>>>>>>>>>>>>>>>> contains
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             multiple rows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join with a record
> >>>>> in
> >>>>>>>> table1
> >>>>>>>>>>>> (what is
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> main
> >>>>>>>>>>>>>>>>>>>>>>>> purpose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > foreign
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join), the result
> >>>>> table
> >>>>>>>>> would
> >>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>> contain a
> >>>>>>>>>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             join result,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> multiple.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Example:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table1 input stream:
> >>>>>>> <A,X>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table2 input stream:
> >>>>>>>>>> <a,(A,1)>,
> >>>>>>>>>>>>>>>>> <b,(A,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> We use table2 value
> >>>>> a
> >>>>>>>>> foreign
> >>>>>>>>>>> key
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> table1
> >>>>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             "A" joins).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result key is the
> >>>>> same
> >>>>>>> key
> >>>>>>>>> as
> >>>>>>>>>>> key
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> table1,
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             implies that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result can either be
> >>>>>>> <A,
> >>>>>>>>>>>> join(X,1)>
> >>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>> <A,
> >>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             but not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > both.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Because the share
> >>>>> the
> >>>>>>> same
> >>>>>>>>>> key,
> >>>>>>>>>>>>>>>>> whatever
> >>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we emit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > later,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> overwrite the
> >>>>> previous
> >>>>>>>>> result.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This is the reason
> >>>>> why
> >>>>>>> Jan
> >>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > combination
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> both primary keys of
> >>>>>>> the
> >>>>>>>>> input
> >>>>>>>>>>>> tables
> >>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             output table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> makes the keys of
> >>>>> the
> >>>>>>>> output
> >>>>>>>>>>> table
> >>>>>>>>>>>>>>>>> unique
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             store both in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> output table:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Result would be
> >>>>> <A-a,
> >>>>>>>>>>> join(X,1)>,
> >>>>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> On 9/4/18 1:36 PM,
> >>>>> Jan
> >>>>>>>>>> Filipiak
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Just on remark here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> The high-watermark
> >>>>>>> could
> >>>>>>>> be
> >>>>>>>>>>>>>>>>> disregarded.
> >>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>> decision
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             about the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> depends on the
> >>>>> size of
> >>>>>>>> the
> >>>>>>>>>>>>>> aggregated
> >>>>>>>>>>>>>>>>>>> map.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> Only 1 element long
> >>>>>>> maps
> >>>>>>>>>> would
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> unpacked
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             forwarded. 0
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > element
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> maps
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> would be published
> >>>>> as
> >>>>>>>>> delete.
> >>>>>>>>>>> Any
> >>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>> count
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of map entries is
> >>>>> in
> >>>>>>>>> "waiting
> >>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> correct
> >>>>>>>>>>>>>>>>>>>>>>>> deletes to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > arrive"-state.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> On 04.09.2018
> >>>>> 21:29,
> >>>>>>> Adam
> >>>>>>>>>>>> Bellemare
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> It does look like I
> >>>>>>> could
> >>>>>>>>>>> replace
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartition store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> highwater store
> >>>>> with
> >>>>>>> a
> >>>>>>>>>> groupBy
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> reduce.
> >>>>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             it looks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> still need to
> >>>>> store
> >>>>>>> the
> >>>>>>>>>>>> highwater
> >>>>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>>>>>>> within
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             materialized
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> compare the
> >>>>> arrival of
> >>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>>>> (assuming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> THIS is
> >>>>> correct...).
> >>>>>>> This
> >>>>>>>>> in
> >>>>>>>>>>>> effect
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> now,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> just with the two
> >>>>>>> tables
> >>>>>>>>>> merged
> >>>>>>>>>>>>>>>>> together.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> --
> >>>>>>>> -- Guozhang
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

I am kinda surprised! Yes my solution of course is correct. Don't really 
know what to show in an example as I am convinced you grabbed the 
concept of how mine works,

If there is a race condition there is a race condition. It doesn't 
matter if there is 10 minutes or milliseconds between events. Either 
they are properly guarded or not. My solution has no such race 
condition. It is 'eventual consistent'. You gonna see all sort of stuff 
coming up during a reprocess.

The user can still fk it up later though. But that is usual business.

In reality I try to supress updates from left sides as long as possible 
because right side updates are more expensive if left is already 
fullish. So that limits the space a little but there are no grantees. 
The result however, after lag is zero is the same every time.

The trade-offs can be shifted as you like. My solution gives full power 
to the user and only does a minimum in the framework. You push 
everything into streams.

If you ask me, not a good choice. Will anyone listen. No.
I do actually think its to late to do my way. It's not like if you 
haven't been gone through the effort and building it.

Just wanted to give you guys another chance, to think it through  ;)

Regarding what will be observed. I consider it a plus that all events 
that are in the inputs have an respective output. Whereas your solution 
might "swallow" events.

Best Jan


On 02.01.2019 15:30, Adam Bellemare wrote:
> Jan
>
> I have been thinking a lot about the history of the discussion and your
> original proposal, and why you believe it is a better solution. The biggest
> problem with your original proposed design is that it seems to me to be
> non-deterministic. It is subject to race conditions that are dependent
> entirely on the data, and without resolution of these races you can end up
> with different results each time. If I am mistaken and this is indeed
> deterministic, then please let me know and provide an explanation, ideally
> with an example.
>
> The way I see it is that you will get very different answers to your
> non-race-condition-resolved join topology, especially if you are nesting it
> with additional joins as you have indicated you are doing. Consider
> rebuilding an application state from the beginning of two topics. If the
> left/this side has multiple foreign-key changes in a row, spaced out every
> ten minutes, you may see something like this:
>
> (foo, foreignKey=red) t=0
> (foo, foreignKey=blue) t=0+10m
> (foo, foreignKey=green) t=0+20m
> (foo, foreignKey=purple) t=0+30m
> (foo, foreignKey=blue) t=0+40m
> (foo, foreignKey=white) t=0+50m
>
> During realtime processing, all of the updates may have correctly
> propagated because it took less than 10 minutes to resolve each join. Upon
> rebuilding from the start, however, all of these events would be processed
> in quick succession. The presence or absence of data will affect the
> results of your join, and the results can vary with each run depending on
> the data. Because of this, I cannot support any kind of solution that would
> allow the exposure of an unresolved intermediate state. I can understand if
> you don't support this, but this is why, as you said, you have the freedom
> to use the Processor API.
>
>
> With that being said, either the solution that I originally proposed
> (join's ocurring on the foreign node) or John + Guozhang's solution
> (registering with the foreign node for notifications) is fine with me -
> both have the same API and we can evaluate it further during implementation.
>
>
> Thanks
>
> Adam
>
> On Thu, Dec 27, 2018 at 2:38 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Hi,
>>
>> just want to let you guys know that this thing is spiralling out of
>> control if you ask me.
>>
>> First you take away the possibility for the user to optimize. Now you
>> pile up complexity to perform some afterwards optimisation, that from my
>> POV completely misses the point. As if the actual call to the joiner
>> really gonna be an expensive part. It wont. Truth is, you don't have a
>> clue which side is gonna be smaller. might be the key you shuffle around
>> is >>> than the value on the other side already.
>>
>> You know my opinion on this. For me its dead, I just leave you the
>> message here as an opportunity to reconsider the choices that were made.
>>
>> Whish y'll a happy new year :)
>>
>>
>>
>>
>>
>>
>> On 27.12.2018 17:22, Adam Bellemare wrote:
>>> Hi All
>>>
>>> Sorry for the delay - holidays and all. I have since updated the KIP with
>>> John's original suggestion and have pruned a number of the no longer
>>> relevant diagrams. Any more comments would be welcomed, otherwise I will
>>> look to kick off the vote again shortly.
>>>
>>> Thanks
>>> Adam
>>>
>>> On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <adam.bellemare@gmail.com
>>>
>>> wrote:
>>>
>>>> Hi John and Guozhang
>>>>
>>>> Ah yes, I lost that in the mix! Thanks for the convergent solutions - I
>> do
>>>> think that the attachment that John included makes for a better design.
>> It
>>>> should also help with overall performance as very high-cardinality
>> foreign
>>>> keyed data (say millions of events with the same entity) will be able to
>>>> leverage the multiple nodes for join functionality instead of having it
>> all
>>>> performed in one node. There is still a bottleneck in the right table
>>>> having to propagate all those events, but with slimmer structures, less
>> IO
>>>> and no need to perform the join I think the throughput will be much
>> higher
>>>> in those scenarios.
>>>>
>>>> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
>>>> version of John's diagram and ensure that the example flow matches
>>>> correctly. Then I can go back to working on the PR to match the diagram.
>>>>
>>>> Thanks both of you for all the help - very much appreciated.
>>>>
>>>> Adam
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>>
>>>>> Hi John,
>>>>>
>>>>> Just made a pass on your diagram (nice hand-drawing btw!), and
>> obviously
>>>>> we
>>>>> are thinking about the same thing :) A neat difference that I like, is
>>>>> that
>>>>> in the pre-join repartition topic we can still send message in the
>> format
>>>>> of `K=k, V=(i=2)` while using "i" as the partition key in
>>>>> StreamsPartition,
>>>>> this way we do not need to even augment the key for the repartition
>> topic,
>>>>> but just do a projection on the foreign key part but trim all other
>>>>> fields:
>>>>> as long as we still materialize the store as `A-2` co-located with the
>>>>> right KTable, that is fine.
>>>>>
>>>>> As I mentioned in my previous email, I also think this has a few
>>>>> advantages
>>>>> on saving over-the-wire bytes as well as disk bytes.
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io>
>> wrote:
>>>>>
>>>>>> Hi Guozhang,
>>>>>>
>>>>>> Thanks for taking a look! I think Adam's already addressed your
>>>>> questions
>>>>>> as well as I could have.
>>>>>>
>>>>>> Hi Adam,
>>>>>>
>>>>>> Thanks for updating the KIP. It looks great, especially how all the
>>>>>> need-to-know information is right at the top, followed by the details.
>>>>>>
>>>>>> Also, thanks for that high-level diagram. Actually, now that I'm
>> looking
>>>>>> at it, I think part of my proposal got lost in translation, although I
>>>>> do
>>>>>> think that what you have there is also correct.
>>>>>>
>>>>>> I sketched up a crude diagram based on yours and attached it to the
>> KIP
>>>>>> (I'm not sure if attached or inline images work on the mailing list):
>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
>>>>>> . It's also attached to this email for convenience.
>>>>>>
>>>>>> Hopefully, you can see how it's intended to line up, and which parts
>> are
>>>>>> modified.
>>>>>> At a high level, instead of performing the join on the right-hand
>> side,
>>>>>> we're essentially just registering interest, like "LHS key A wishes to
>>>>>> receive updates for RHS key 2". Then, when there is a new "interest"
>> or
>>>>> any
>>>>>> updates to the RHS records, it "broadcasts" its state back to the LHS
>>>>>> records who are interested in it.
>>>>>>
>>>>>> Thus, instead of sending the LHS values to the RHS joiner workers and
>>>>> then
>>>>>> sending the join results back to the LHS worke be co-partitioned and
>>>>>> validated, we instead only send the LHS *keys* to the RHS workers and
>>>>> then
>>>>>> only the RHS k/v back to be joined by the LHS worker.
>>>>>>
>>>>>> I've been considering both your diagram and mine, and I *think* what
>> I'm
>>>>>> proposing has a few advantages.
>>>>>>
>>>>>> Here are some points of interest as you look at the diagram:
>>>>>> * When we extract the foreign key and send it to the Pre-Join
>>>>> Repartition
>>>>>> Topic, we can send only the FK/PK pair. There's no need to worry about
>>>>>> custom partitioner logic, since we can just use the foreign key
>> plainly
>>>>> as
>>>>>> the repartition record key. Also, we save on transmitting the LHS
>> value,
>>>>>> since we only send its key in this step.
>>>>>> * We also only need to store the RHSKey:LHSKey mapping in the
>>>>>> MaterializedSubscriptionStore, saving on disk. We can use the same
>> rocks
>>>>>> key format you proposed and the same algorithm involving range scans
>>>>> when
>>>>>> the RHS records get updated.
>>>>>> * Instead of joining on the right side, all we do is compose a
>>>>>> re-repartition record so we can broadcast the RHS k/v pair back to the
>>>>>> original LHS partition. (this is what the "rekey" node is doing)
>>>>>> * Then, there is a special kind of Joiner that's co-resident in the
>> same
>>>>>> StreamTask as the LHS table, subscribed to the Post-Join Repartition
>>>>> Topic.
>>>>>> ** This Joiner is *not* triggered directly by any changes in the LHS
>>>>>> KTable. Instead, LHS events indirectly trigger the join via the whole
>>>>>> lifecycle.
>>>>>> ** For each event arriving from the Post-Join Repartition Topic, the
>>>>>> Joiner looks up the corresponding record in the LHS KTable. It
>> validates
>>>>>> the FK as you noted, discarding any inconsistent events. Otherwise, it
>>>>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the
>> join
>>>>>> result
>>>>>> ** Note that the Joiner itself is stateless, so materializing the join
>>>>>> result is optional, just as with the 1:1 joins.
>>>>>>
>>>>>> So in summary:
>>>>>> * instead of transmitting the LHS keys and values to the right and the
>>>>>> JoinResult back to the left, we only transmit the LHS keys to the
>> right
>>>>> and
>>>>>> the RHS values to the left. Assuming the average RHS value is on
>> smaller
>>>>>> than or equal to the average join result size, it's a clear win on
>>>>> broker
>>>>>> traffic. I think this is actually a reasonable assumption, which we
>> can
>>>>>> discuss more if you're suspicious.
>>>>>> * we only need one copy of the data (the left and right tables need to
>>>>> be
>>>>>> materialized) and one extra copy of the PK:FK pairs in the
>> Materialized
>>>>>> Subscription Store. Materializing the join result is optional, just as
>>>>> with
>>>>>> the existing 1:1 joins.
>>>>>> * we still need the fancy range-scan algorithm on the right to locate
>>>>> all
>>>>>> interested LHS keys when a RHS value is updated, but we don't need a
>>>>> custom
>>>>>> partitioner for either repartition topic (this is of course a
>>>>> modification
>>>>>> we could make to your version as well)
>>>>>>
>>>>>> How does this sound to you? (And did I miss anything?)
>>>>>> -John
>>>>>>
>>>>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
>>>>> adam.bellemare@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi John & Guozhang
>>>>>>>
>>>>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the
>>>>> KIP,
>>>>>>> pruned much of what I wrote and put a simplified diagram near the top
>>>>> to
>>>>>>> illustrate the workflow. I encapsulated Jan's content at the bottom
>> of
>>>>> the
>>>>>>> document. I believe it is simpler to read by far now.
>>>>>>>
>>>>>>> @Guozhang Wang <wa...@gmail.com>:
>>>>>>>> #1: rekey left table
>>>>>>>>     -> source from the left upstream, send to rekey-processor to
>>>>> generate
>>>>>>> combined key, and then sink to copartition topic.
>>>>>>> Correct.
>>>>>>>
>>>>>>>> #2: first-join with right table
>>>>>>>>     -> source from the right table upstream, materialize the right
>>>>> table.
>>>>>>>>     -> source from the co-partition topic, materialize the rekeyed
>> left
>>>>>>> table, join with the right table, rekey back, and then sink to the
>>>>>>> rekeyed-back topic.
>>>>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I need
>> the
>>>>>>> Foreign-Key value generated in #1 above to compare in the resolution
>>>>>>> stage.
>>>>>>>
>>>>>>>> #3: second join
>>>>>>>>      -> source from the rekeyed-back topic, materialize the rekeyed
>>>>> back
>>>>>>> table.
>>>>>>>>     -> source from the left upstream, materialize the left table,
>> join
>>>>>>> with
>>>>>>> the rekeyed back table.
>>>>>>> Almost - As each event comes in, we just run it through a stateful
>>>>>>> processor that checks the original ("This") KTable for the key. The
>>>>> value
>>>>>>> payload then has the foreignKeyExtractor applied again as in Part #1
>>>>>>> above,
>>>>>>> and gets the current foreign key. Then we compare it to the joined
>>>>> event
>>>>>>> that we are currently resolving. If they have the same foreign-key,
>>>>>>> propagate the result out. If they don't, throw the event away.
>>>>>>>
>>>>>>> The end result is that we do need to materialize 2 additional tables
>>>>>>> (left/this-combinedkey table, and the final Joined table) as I've
>>>>>>> illustrated in the updated KIP. I hope the diagram clears it up a lot
>>>>>>> better. Please let me know.
>>>>>>>
>>>>>>> Thanks again
>>>>>>> Adam
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>>>
>>>>>>>> John,
>>>>>>>>
>>>>>>>> Thanks a lot for the suggestions on refactoring the wiki, I agree
>>>>> with
>>>>>>> you
>>>>>>>> that we should consider the KIP proposal to be easily understood by
>>>>>>> anyone
>>>>>>>> in the future to read, and hence should provide a good summary on
>> the
>>>>>>>> user-facing interfaces, as well as rejected alternatives to
>> represent
>>>>>>>> briefly "how we came a long way to this conclusion, and what we have
>>>>>>>> argued, disagreed, and agreed about, etc" so that readers do not
>>>>> need to
>>>>>>>> dig into the DISCUSS thread to get all the details. We can, of
>>>>> course,
>>>>>>> keep
>>>>>>>> the implementation details like "workflows" on the wiki page as a
>>>>>>> addendum
>>>>>>>> section since it also has correlations.
>>>>>>>>
>>>>>>>> Regarding your proposal on comment 6): that's a very interesting
>>>>> idea!
>>>>>>> Just
>>>>>>>> to clarify that I understands it fully correctly: the proposal's
>>>>>>> resulted
>>>>>>>> topology is still the same as the current proposal, where we will
>>>>> have 3
>>>>>>>> sub-topologies for this operator:
>>>>>>>>
>>>>>>>> #1: rekey left table
>>>>>>>>      -> source from the left upstream, send to rekey-processor to
>>>>> generate
>>>>>>>> combined key, and then sink to copartition topic.
>>>>>>>>
>>>>>>>> #2: first-join with right table
>>>>>>>>      -> source from the right table upstream, materialize the right
>>>>> table.
>>>>>>>>      -> source from the co-partition topic, materialize the rekeyed
>>>>> left
>>>>>>>> table, join with the right table, rekey back, and then sink to the
>>>>>>>> rekeyed-back topic.
>>>>>>>>
>>>>>>>> #3: second join
>>>>>>>>      -> source from the rekeyed-back topic, materialize the rekeyed
>>>>> back
>>>>>>>> table.
>>>>>>>>      -> source from the left upstream, materialize the left table,
>> join
>>>>>>> with
>>>>>>>> the rekeyed back table.
>>>>>>>>
>>>>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology since
>>>>>>> both of
>>>>>>>> them read from the left table source stream. In this workflow, we
>>>>> need
>>>>>>> to
>>>>>>>> materialize 4 tables (left table in #3, right table in #2, rekeyed
>>>>> left
>>>>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
>>>>>>>> (copartition topic, rekeyed-back topic).
>>>>>>>>
>>>>>>>> Compared with Adam's current proposal in the workflow overview, it
>>>>> has
>>>>>>> the
>>>>>>>> same num.materialize tables (left table, rekeyed left table, right
>>>>>>> table,
>>>>>>>> out-of-ordering resolver table), and same num.internal topics (two).
>>>>> The
>>>>>>>> advantage is that on the copartition topic, we can save bandwidth by
>>>>> not
>>>>>>>> sending value, and in #2 the rekeyed left table is smaller since we
>>>>> do
>>>>>>> not
>>>>>>>> have any values to materialize. Is that right?
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Adam,
>>>>>>>>>
>>>>>>>>> Given that the committers are all pretty busy right now, I think
>>>>> that
>>>>>>> it
>>>>>>>>> would help if you were to refactor the KIP a little to reduce the
>>>>>>>> workload
>>>>>>>>> for reviewers.
>>>>>>>>>
>>>>>>>>> I'd recommend the following changes:
>>>>>>>>> * relocate all internal details to a section at the end called
>>>>>>> something
>>>>>>>>> like "Implementation Notes" or something like that.
>>>>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
>>>>> mention
>>>>>>>> only
>>>>>>>>> publicly-facing API changes.
>>>>>>>>> ** for example, the interface that you've already listed there, as
>>>>>>> well
>>>>>>>> as
>>>>>>>>> a textual description of the guarantees we'll be providing (join
>>>>>>> result
>>>>>>>> is
>>>>>>>>> copartitioned with the LHS, and the join result is guaranteed
>>>>> correct)
>>>>>>>>>
>>>>>>>>> A good target would be that the whole main body of the KIP,
>>>>> including
>>>>>>>>> Status, Motivation, Proposal, Justification, and Rejected
>>>>> Alternatives
>>>>>>>> all
>>>>>>>>> fit "above the fold" (i.e., all fit on the screen at a comfortable
>>>>>>> zoom
>>>>>>>>> level).
>>>>>>>>> I think the only real Rejected Alternative that bears mention at
>>>>> this
>>>>>>>> point
>>>>>>>>> is KScatteredTable, which you could just include the executive
>>>>>>> summary on
>>>>>>>>> (no implementation details), and link to extra details in the
>>>>>>>>> Implementation Notes section.
>>>>>>>>>
>>>>>>>>> Taking a look at the wiki page, ~90% of the text there is internal
>>>>>>>> detail,
>>>>>>>>> which is useful for the dubious, but doesn't need to be ratified
>>>>> in a
>>>>>>>> vote
>>>>>>>>> (and would be subject to change without notice in the future
>>>>> anyway).
>>>>>>>>> There's also a lot of conflicting discussion, as you've very
>>>>>>> respectfully
>>>>>>>>> tried to preserve the original proposal from Jan while adding your
>>>>>>> own.
>>>>>>>>> Isolating all this information in a dedicated section at the bottom
>>>>>>> frees
>>>>>>>>> the voters up to focus on the public API part of the proposal,
>>>>> which
>>>>>>> is
>>>>>>>>> really all they need to consider.
>>>>>>>>>
>>>>>>>>> Plus, it'll be clear to future readers which parts of the document
>>>>> are
>>>>>>>>> enduring, and which parts are a snapshot of our implementation
>>>>>>> thinking
>>>>>>>> at
>>>>>>>>> the time.
>>>>>>>>>
>>>>>>>>> I'm suggesting this because I suspect that the others haven't made
>>>>>>> time
>>>>>>>> to
>>>>>>>>> review it partly because it seems daunting. If it seems like it
>>>>> would
>>>>>>> be
>>>>>>>> a
>>>>>>>>> huge time investment to review, people will just keep putting it
>>>>> off.
>>>>>>> But
>>>>>>>>> if the KIP is a single page, then they'll be more inclined to give
>>>>> it
>>>>>>> a
>>>>>>>>> read.
>>>>>>>>>
>>>>>>>>> Honestly, I don't think the KIP itself is that controversial (apart
>>>>>>> from
>>>>>>>>> the scattered table thing (sorry, Jan) ). Most of the discussion
>>>>> has
>>>>>>> been
>>>>>>>>> around the implementation, which we can continue more effectively
>>>>> in
>>>>>>> a PR
>>>>>>>>> once the KIP has passed.
>>>>>>>>>
>>>>>>>>> How does that sound?
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
>>>>>>> adam.bellemare@gmail.com
>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 1) I believe that the resolution mechanism John has proposed is
>>>>>>>>> sufficient
>>>>>>>>>> - it is clean and easy and doesn't require additional RocksDB
>>>>>>> stores,
>>>>>>>>> which
>>>>>>>>>> reduces the footprint greatly. I don't think we need to resolve
>>>>>>> based
>>>>>>>> on
>>>>>>>>>> timestamp or offset anymore, but if we decide to do to that
>>>>> would be
>>>>>>>>> within
>>>>>>>>>> the bounds of the existing API.
>>>>>>>>>>
>>>>>>>>>> 2) Is the current API sufficient, or does it need to be altered
>>>>> to
>>>>>>> go
>>>>>>>>> back
>>>>>>>>>> to vote?
>>>>>>>>>>
>>>>>>>>>> 3) KScatteredTable implementation can always be added in a future
>>>>>>>>> revision.
>>>>>>>>>> This API does not rule it out. This implementation of this
>>>>> function
>>>>>>>> would
>>>>>>>>>> simply be replaced with `KScatteredTable.resolve()` while still
>>>>>>>>> maintaining
>>>>>>>>>> the existing API, thereby giving both features as Jan outlined
>>>>>>> earlier.
>>>>>>>>>> Would this work?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks Guozhang, John and Jan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <john@confluent.io
>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi, all,
>>>>>>>>>>>
>>>>>>>>>>>>> In fact, we
>>>>>>>>>>>>> can just keep a single final-result store with timestamps
>>>>> and
>>>>>>>> reject
>>>>>>>>>>> values
>>>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>>>
>>>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>>>> offset
>>>>>>>>> of
>>>>>>>>>>>> the original message.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for this point, Jan.
>>>>>>>>>>>
>>>>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
>>>>> the
>>>>>>> k/v
>>>>>>>>>>> store,
>>>>>>>>>>> as well as providing a storage-format upgrade path.
>>>>>>>>>>>
>>>>>>>>>>> I might have missed it, but I think we have yet to discuss
>>>>> whether
>>>>>>>> it's
>>>>>>>>>>> safe
>>>>>>>>>>> or desirable just to swap topic-ordering our for
>>>>>>> timestamp-ordering.
>>>>>>>>> This
>>>>>>>>>>> is
>>>>>>>>>>> a very deep topic, and I think it would only pollute the
>>>>> current
>>>>>>>>>>> discussion.
>>>>>>>>>>>
>>>>>>>>>>> What Adam has proposed is safe, given the *current* ordering
>>>>>>>> semantics
>>>>>>>>>>> of the system. If we can agree on his proposal, I think we can
>>>>>>> merge
>>>>>>>>> the
>>>>>>>>>>> feature well before the conversation about timestamp ordering
>>>>> even
>>>>>>>>> takes
>>>>>>>>>>> place, much less reaches a conclusion. In the mean time, it
>>>>> would
>>>>>>>> seem
>>>>>>>>> to
>>>>>>>>>>> be unfortunate to have one join operator with different
>>>>> ordering
>>>>>>>>>> semantics
>>>>>>>>>>> from every other KTable operator.
>>>>>>>>>>>
>>>>>>>>>>> If and when that timestamp discussion takes place, many (all?)
>>>>>>> KTable
>>>>>>>>>>> operations
>>>>>>>>>>> will need to be updated, rendering the many:one join a small
>>>>>>> marginal
>>>>>>>>>> cost.
>>>>>>>>>>>
>>>>>>>>>>> And, just to plug it again, I proposed an algorithm above that
>>>>> I
>>>>>>>>> believe
>>>>>>>>>>> provides
>>>>>>>>>>> correct ordering without any additional metadata, and
>>>>> regardless
>>>>>>> of
>>>>>>>> the
>>>>>>>>>>> ordering semantics. I didn't bring it up further, because I
>>>>> felt
>>>>>>> the
>>>>>>>>> KIP
>>>>>>>>>>> only needs
>>>>>>>>>>> to agree on the public API, and we can discuss the
>>>>> implementation
>>>>>>> at
>>>>>>>>>>> leisure in
>>>>>>>>>>> a PR...
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
>>>>>>>>>>>>> Hello Adam / Jan / John,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Sorry for being late on this thread! I've finally got some
>>>>>>> time
>>>>>>>>> this
>>>>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
>>>>> I've
>>>>>>>> also
>>>>>>>>>>>> realized
>>>>>>>>>>>>> there are a bunch of other things I need to enqueue while
>>>>>>>> cleaning
>>>>>>>>>> them
>>>>>>>>>>>> up
>>>>>>>>>>>>> --- sth I need to improve on my side). So here are my
>>>>>>> thoughts:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regarding the APIs: I like the current written API in the
>>>>> KIP.
>>>>>>>> More
>>>>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
>>>>>>>>> functionalities
>>>>>>>>>> as
>>>>>>>>>>>>> well as 2) other join types than inner as separate KIPs
>>>>> since
>>>>>>> 1)
>>>>>>>>> may
>>>>>>>>>>>> worth
>>>>>>>>>>>>> a general API refactoring that can benefit not only
>>>>> foreignkey
>>>>>>>>> joins
>>>>>>>>>>> but
>>>>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
>>>>>>>>>>>> ),
>>>>>>>>>>>>> and I'm not sure if other join types would actually be
>>>>> needed
>>>>>>>>> (maybe
>>>>>>>>>>> left
>>>>>>>>>>>>> join still makes sense), so it's better to
>>>>>>>>>>> wait-for-people-to-ask-and-add
>>>>>>>>>>>>> than add-sth-that-no-one-uses.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
>>>>> a
>>>>>>>>>>>>> KScatteredTable for users to inject their own optimization:
>>>>>>> I'd
>>>>>>>>>> prefer
>>>>>>>>>>> to
>>>>>>>>>>>>> do the current option as-is, and my main rationale is for
>>>>>>>>>> optimization
>>>>>>>>>>>>> rooms inside the Streams internals and the API
>>>>> succinctness.
>>>>>>> For
>>>>>>>>>>> advanced
>>>>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
>>>>> own
>>>>>>>>>>>> optimization,
>>>>>>>>>>>>> while it is too much of the work to use Processor API
>>>>>>> directly, I
>>>>>>>>>> think
>>>>>>>>>>>> we
>>>>>>>>>>>>> can still extend the current API to support it in the
>>>>> future
>>>>>>> if
>>>>>>>> it
>>>>>>>>>>>> becomes
>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>
>>>>>>>>>>>> no internal optimization potential. it's a myth
>>>>>>>>>>>>
>>>>>>>>>>>> ¯\_(ツ)_/¯
>>>>>>>>>>>>
>>>>>>>>>>>> :-)
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
>>>>> as
>>>>>>> I
>>>>>>>>>>> mentioned
>>>>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
>>>>> key-value
>>>>>>>>> store)
>>>>>>>>>>> we
>>>>>>>>>>>>> can actually make this step simpler than the current
>>>>>>> proposal. In
>>>>>>>>>> fact,
>>>>>>>>>>>> we
>>>>>>>>>>>>> can just keep a single final-result store with timestamps
>>>>> and
>>>>>>>>> reject
>>>>>>>>>>>> values
>>>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>>>>
>>>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>>>> offset
>>>>>>>>> of
>>>>>>>>>>>> the original message.
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
>>>>>>> Adam
>>>>>>>> to
>>>>>>>>>>> make
>>>>>>>>>>>>> such HUGE progress on this KIP!
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> If they don't find the time:
>>>>>>>>>>>>>> They usually take the opposite path from me :D
>>>>>>>>>>>>>> so the answer would be clear.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> hence my suggestion to vote.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
>>>>>>>>>>>>>>> Hi Guozhang and Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
>>>>> KIP
>>>>>>>> to a
>>>>>>>>>>> point
>>>>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
>>>>> a
>>>>>>>>>>> tie-breaker,
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> you will). If you have anyone else you may think should
>>>>>>> look at
>>>>>>>>>> this,
>>>>>>>>>>>>>>> please tag them accordingly.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The scenario is as such:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Current Option:
>>>>>>>>>>>>>>> API:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>>>>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
>>>>>>>>> partition
>>>>>>>>>>> with
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> foreignKey (repartition 1)
>>>>>>>>>>>>>>> 2) Join the data
>>>>>>>>>>>>>>> 3) Shuffle the data back to the original node
>>>>> (repartition
>>>>>>> 2)
>>>>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
>>>>>>>>> foreign-key
>>>>>>>>>>>>>> changes.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Alternate Option:
>>>>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
>>>>>>>>>>>>>>> - It would be keyed on a wrapped key function:
>>>>>>> <CombinedKey<KO,
>>>>>>>>> K>,
>>>>>>>>>>> VR>
>>>>>>>>>>>>>> (KO
>>>>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
>>>>> Result)
>>>>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
>>>>>>>>> otherwise a
>>>>>>>>>>>> user
>>>>>>>>>>>>>>> would be able to perform additional functions directly
>>>>> from
>>>>>>> the
>>>>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
>>>>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
>>>>>>> tradeoffs.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
>>>>> possible,
>>>>>>>> but
>>>>>>>>>> will
>>>>>>>>>>>>>>> require for implementation details to be made in the API
>>>>> and
>>>>>>>> some
>>>>>>>>>>>>>> exposure
>>>>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I appreciate any insight into this.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
>>>>>>> summary
>>>>>>>> is
>>>>>>>>>>>>>> accurate
>>>>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
>>>>> that
>>>>>>>>> there
>>>>>>>>>>> is a
>>>>>>>>>>>>>> risk
>>>>>>>>>>>>>>>> of inconsistent final states without performing the
>>>>>>>> resolution.
>>>>>>>>>> This
>>>>>>>>>>>> is
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> major concern for me as most of the data I have dealt
>>>>> with
>>>>>>> is
>>>>>>>>>>> produced
>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>> relational databases. We have seen a number of cases
>>>>> where
>>>>>>> a
>>>>>>>>> user
>>>>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
>>>>>>> they
>>>>>>>>> made
>>>>>>>>>> a
>>>>>>>>>>>>>>>> mistake, and then updated the field again with a new
>>>>> key.
>>>>>>> The
>>>>>>>>>> events
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> propagated out as they are produced, and as such we have
>>>>>>> had
>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>> cases where these inconsistencies were propagated
>>>>>>> downstream
>>>>>>>> as
>>>>>>>>>> the
>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>> values due to the race conditions in the fanout of the
>>>>>>> data.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This solution that I propose values correctness of the
>>>>>>> final
>>>>>>>>>> result
>>>>>>>>>>>> over
>>>>>>>>>>>>>>>> other factors.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We could always move this function over to using a
>>>>>>>>> KScatteredTable
>>>>>>>>>>>>>>>> implementation in the future, and simply deprecate it
>>>>> this
>>>>>>>> join
>>>>>>>>>> API
>>>>>>>>>>> in
>>>>>>>>>>>>>>>> time. I think I would like to hear more from some of the
>>>>>>> other
>>>>>>>>>> major
>>>>>>>>>>>>>>>> committers on which course of action they would think is
>>>>>>> best
>>>>>>>>>> before
>>>>>>>>>>>> any
>>>>>>>>>>>>>>>> more coding is done.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks again
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
>>>>>>>> john@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Jan and Adam,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
>>>>> are
>>>>>>>>>>> encouraging.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
>>>>>>> agree
>>>>>>>>> that
>>>>>>>>>>>>>> avoiding
>>>>>>>>>>>>>>>>> unnecessary join outputs is especially important when
>>>>> the
>>>>>>>>> fan-out
>>>>>>>>>>> is
>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>> high. I suppose this could also be built into the
>>>>>>>>> implementation
>>>>>>>>>>>> we're
>>>>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
>>>>>>> KIP
>>>>>>>>>> (since
>>>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>> API-transparent optimization).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
>>>>>>> didn't
>>>>>>>>>> bring
>>>>>>>>>>>> it
>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
>>>>> the
>>>>>>>> KIP
>>>>>>>>>>> as-is,
>>>>>>>>>>>>>>>>> despite the disagreement.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
>>>>>>>>>> reasonable.
>>>>>>>>>>>>>>>>> It sounds like Jan values more the potential for
>>>>>>> developers
>>>>>>>> to
>>>>>>>>>>>> optimize
>>>>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
>>>>> whereas
>>>>>>>> Adam
>>>>>>>>>>>> places
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> value on having a single operator that people can use
>>>>>>> without
>>>>>>>>>> extra
>>>>>>>>>>>>>> steps
>>>>>>>>>>>>>>>>> at the end.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Personally, although I do find it exceptionally
>>>>> annoying
>>>>>>>> when a
>>>>>>>>>>>>>> framework
>>>>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
>>>>> it
>>>>>>>> seems
>>>>>>>>>>> better
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>> for a single operation.
>>>>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
>>>>>>> significant
>>>>>>>>>>>> latitude
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> the implementation (for example, joining only at the
>>>>> end,
>>>>>>> not
>>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> middle
>>>>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
>>>>> resolution;
>>>>>>> how
>>>>>>>> we
>>>>>>>>>>>>>> represent
>>>>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
>>>>>>> vectors),
>>>>>>>>>>> etc.).
>>>>>>>>>>>>>> If we
>>>>>>>>>>>>>>>>> publish something like a KScatteredTable with the
>>>>>>>>>> right-partitioned
>>>>>>>>>>>>>> joined
>>>>>>>>>>>>>>>>> data, then the API pretty much locks in the
>>>>>>> implementation as
>>>>>>>>>> well.
>>>>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
>>>>> mean
>>>>>>>>> "seems";
>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> anyone
>>>>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
>>>>>>>>> simpler,
>>>>>>>>>> I
>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
>>>>>>> algebra
>>>>>>>>>>>>>> perspective,
>>>>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
>>>>>>> KTable
>>>>>>>>> in
>>>>>>>>>>> all
>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>> * That said, there might still be room in the API for a
>>>>>>>>> different
>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
>>>>> then
>>>>>>> do
>>>>>>>>>> things
>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
>>>>> haven't
>>>>>>>>> thought
>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>> all the consequences yet.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This is all just my opinion after thinking over the
>>>>>>>> discussion
>>>>>>>>> so
>>>>>>>>>>>>>> far...
>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
>>>>> the
>>>>>>>>>>>> prefixScan.
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> have attached the file, but I will also include the
>>>>> text
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>> body
>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
>>>>>>>> attached
>>>>>>>>>>>> files).
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
>>>>> large
>>>>>>>>> values
>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
>>>>>>> would
>>>>>>>> be
>>>>>>>>>> more
>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
>>>>>>> For
>>>>>>>>>>> instance,
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
>>>>> and a
>>>>>>>>> further
>>>>>>>>>>>>>>>>> ~35-85mS
>>>>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
>>>>>>> them.
>>>>>>>>>>> Iterating
>>>>>>>>>>>>>>>>>> through the records just to generate a simple count
>>>>>>> takes ~
>>>>>>>> 40
>>>>>>>>>>> times
>>>>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>>>>> than the flush + scan combined.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>> Setup:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
>>>>>>>>> (Xmx512m,
>>>>>>>>>>>>>> Xms512m)
>>>>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
>>>>> directly-accessible
>>>>>>>>> Kafka
>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
>>>>> the
>>>>>>>>>>>>>>>>>> ProcessorContext).
>>>>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
>>>>> values
>>>>>>>>>> provided
>>>>>>>>>>> in
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> 2.1/trunk release.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> keysize = 128 bytes
>>>>>>>>>>>>>>>>>> valsize = 512 bytes
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Step 1:
>>>>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
>>>>>>>> left-padded
>>>>>>>>>>>>>>>>>> auto-incrementing integer)
>>>>>>>>>>>>>>>>>> Step 2:
>>>>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
>>>>>>>>>>>>>> auto-incrementing
>>>>>>>>>>>>>>>>>> integer)
>>>>>>>>>>>>>>>>>> Step 3:
>>>>>>>>>>>>>>>>>> Perform flush
>>>>>>>>>>>>>>>>>> Step 4:
>>>>>>>>>>>>>>>>>> Perform prefixScan
>>>>>>>>>>>>>>>>>> Step 5:
>>>>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
>>>>> count of
>>>>>>>>>> expected
>>>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>> Results:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> ============================================================================================
>>>>>>>>>>>>>>>>>> X = 1k (11k events total)
>>>>>>>>>>>>>>>>>> Flush Time = 39 mS
>>>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>>>> 6.9 MB disk
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>> X = 10k (110k events total)
>>>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>>>> Scan Time = 8 mS
>>>>>>>>>>>>>>>>>> 127 MB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>> X = 100k (1.1M events total)
>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>> Flush Time = 60 mS
>>>>>>>>>>>>>>>>>> Scan Time = 12 mS
>>>>>>>>>>>>>>>>>> 678 MB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>>>> 576 MB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>> X = 1MB (11M events total)
>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>> Flush Time = 52 mS
>>>>>>>>>>>>>>>>>> Scan Time = 19 mS
>>>>>>>>>>>>>>>>>> 7.2 GB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>> Flush Time = 84 mS
>>>>>>>>>>>>>>>>>> Scan Time = 34 mS
>>>>>>>>>>>>>>>>>> 9.1 GB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
>>>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>>>> Flush Time = 82 mS
>>>>>>>>>>>>>>>>>> Scan Time = 63 mS
>>>>>>>>>>>>>>>>>> 17GB - 276 sst files
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>>>> Flush Time = 116 mS
>>>>>>>>>>>>>>>>>> Scan Time = 35 mS
>>>>>>>>>>>>>>>>>> 23GB - 361 sst files
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Test3:
>>>>>>>>>>>>>>>>>> Flush Time = 103 mS
>>>>>>>>>>>>>>>>>> Scan Time = 82 mS
>>>>>>>>>>>>>>>>>> 19 GB - 300 sst files
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
>>>>>>> events. I
>>>>>>>>>> tried
>>>>>>>>>>>> to
>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
>>>>> the
>>>>>>>> 100GB+
>>>>>>>>>>> range
>>>>>>>>>>>>>>>>> and my
>>>>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
>>>>> be
>>>>>>> done
>>>>>>>>>> but I
>>>>>>>>>>>>>>>>> suspect
>>>>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
>>>>>>>> results
>>>>>>>>>>>> above.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> At this point in time, I think the only major
>>>>> discussion
>>>>>>>> point
>>>>>>>>>> is
>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
>>>>> repartitioning
>>>>>>>> back +
>>>>>>>>>>>>>> resolving
>>>>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
>>>>> the
>>>>>>>> client
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks folks,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
>>>>> the
>>>>>>> 2.1
>>>>>>>>>>> release
>>>>>>>>>>>>>>>>>>> caused an
>>>>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
>>>>>>> radar
>>>>>>>>>> (which
>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
>>>>>>> some
>>>>>>>>>> extend
>>>>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
>>>>>>> keep) me
>>>>>>>>>>>>>>>>> preoccupied.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
>>>>> guess
>>>>>>> Jan
>>>>>>>>> was
>>>>>>>>>>> on
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>> track!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
>>>>> and
>>>>>>> the
>>>>>>>>>> prior
>>>>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> API Thoughts:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
>>>>>>>> many-to-one
>>>>>>>>>>> join.
>>>>>>>>>>>>>>>>> Could
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
>>>>>>> flip
>>>>>>>>> the
>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
>>>>> join
>>>>>>>>> type,
>>>>>>>>>>> and
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
>>>>>>>>> one-to-one
>>>>>>>>>>>> join.
>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>> would work, of course, but it would be super
>>>>>>> inefficient
>>>>>>>>>>> compared
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> simple rekey-and-join.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
>>>>>>>> specified
>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
>>>>>>> join, as
>>>>>>>>>>>>>>>>> (neglecting
>>>>>>>>>>>>>>>>>>> IQ),
>>>>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
>>>>>>> it in
>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> ValueJoiner.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
>>>>> quite
>>>>>>>>> right.
>>>>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
>>>>>>>>> paradigms
>>>>>>>>>> in
>>>>>>>>>>>>>>>>> play in
>>>>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
>>>>>>> three
>>>>>>>>>>>> Serialized
>>>>>>>>>>>>>>>>>>> args, I
>>>>>>>>>>>>>>>>>>>> think it would be better to have one that allows
>>>>>>>>> (optionally)
>>>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>>>> the 4
>>>>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
>>>>>>>>>>> Materialized.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
>>>>>>> already
>>>>>>>>> be
>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
>>>>>>> config
>>>>>>>>>> might
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> applicable.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
>>>>> joins
>>>>>>>>> don't
>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
>>>>>>> since
>>>>>>>>> the
>>>>>>>>>>>> rekey
>>>>>>>>>>>>>>>>>>>> operation needs to produce results that are
>>>>>>> co-partitioned
>>>>>>>>>> with
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> "other"
>>>>>>>>>>>>>>>>>>>> KTable.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
>>>>>>> didn't
>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
>>>>>>>>>> "deserializing"
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
>>>>>>> think
>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
>>>>>>> namespace
>>>>>>>> at
>>>>>>>>>>> source
>>>>>>>>>>>>>>>>> nodes
>>>>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
>>>>>>> practically
>>>>>>>> no
>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
>>>>> API
>>>>>>>> change
>>>>>>>>>> wrt
>>>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
>>>>> placed
>>>>>>> on
>>>>>>>>>> users'
>>>>>>>>>>>>>>>>>>> headers.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
>>>>> the
>>>>>>>>> header
>>>>>>>>>> at
>>>>>>>>>>>>>>>>> all...
>>>>>>>>>>>>>>>>>>>> stay tuned)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
>>>>>>> table
>>>>>>>>>>> growing
>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
>>>>>>>>> implementing
>>>>>>>>>>> OCC
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
>>>>> the
>>>>>>>>> rekey
>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
>>>>> changes.
>>>>>>> As
>>>>>>>>> such,
>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
>>>>> offset in
>>>>>>>> the
>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
>>>>> the
>>>>>>> same
>>>>>>>>>>> number
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> keys
>>>>>>>>>>>>>>>>>>>> as the source table at all times.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
>>>>>>> might
>>>>>>>> be
>>>>>>>>>>>> relevant
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
>>>>>>> timestamp
>>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
>>>>> the
>>>>>>>>>> timestamp
>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> OCC
>>>>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
>>>>>>> want to
>>>>>>>>> use
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
>>>>> use a
>>>>>>>>> similar
>>>>>>>>>>>>>>>>>>> mechanism to
>>>>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
>>>>>>>> re-keyed
>>>>>>>>>>>> values,
>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>> you could avoid a separate table.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
>>>>> long
>>>>>>>>> time,
>>>>>>>>>> so
>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
>>>>> if we
>>>>>>>> can
>>>>>>>>>>> avoid
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> HWM
>>>>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
>>>>> final
>>>>>>>> join
>>>>>>>>>>>>>>>>> instead...
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
>>>>> Letter
>>>>>>>> FK,
>>>>>>>>>>> (other
>>>>>>>>>>>>>>>>>>> data))
>>>>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>>>> 1: (A, xyz)
>>>>>>>>>>>>>>>>>>>> 2: (B, asd)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Right table:
>>>>>>>>>>>>>>>>>>>> A: EntityA
>>>>>>>>>>>>>>>>>>>> B: EntityB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
>>>>>>> key,
>>>>>>>> but
>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> propagating the value at all..
>>>>>>>>>>>>>>>>>>>> Rekey table:
>>>>>>>>>>>>>>>>>>>> A-1: (dummy value)
>>>>>>>>>>>>>>>>>>>> B-2: (dummy value)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
>>>>>>>>>>>>>>>>>>>> A-1: EntityA
>>>>>>>>>>>>>>>>>>>> B-2: EntityB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Which gets rekeyed back:
>>>>>>>>>>>>>>>>>>>> 1: A, EntityA
>>>>>>>>>>>>>>>>>>>> 2: B, EntityB
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> And finally we do the actual join:
>>>>>>>>>>>>>>>>>>>> Result table:
>>>>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
>>>>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
>>>>>>>> opportunity
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> compare
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
>>>>>>> the
>>>>>>>>> right
>>>>>>>>>>>>>>>>> table. If
>>>>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
>>>>>>> must be
>>>>>>>>>>>> outdated.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
>>>>> xyz)
>>>>>>> gets
>>>>>>>>>>> updated
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> (1:
>>>>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
>>>>> whether
>>>>>>> the
>>>>>>>>>> final
>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
>>>>>>>> algorithm
>>>>>>>>>>> above,
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
>>>>>>> (B,
>>>>>>>>> xyz),
>>>>>>>>>>> (B,
>>>>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
>>>>>>>>> information
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
>>>>>>> mentioned
>>>>>>>>>> often
>>>>>>>>>>> I
>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
>>>>>>>>> automatically.
>>>>>>>>>> I
>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>> this is the real performance eater ;)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
>>>>>>>>>> performance
>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> range scans when records change in the right table.
>>>>>>> You've
>>>>>>>>>> said
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> you've
>>>>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
>>>>>>> for a
>>>>>>>>>>> while.
>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
>>>>>>> you've
>>>>>>>>>>>> observed?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
>>>>>>> topmost
>>>>>>>>>> thing
>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
>>>>>>>>> performance
>>>>>>>>>>> of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
>>>>>>> 500k -
>>>>>>>>> 1kk
>>>>>>>>>>> rows
>>>>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
>>>>> the
>>>>>>> work
>>>>>>>>> you
>>>>>>>>>>>> gotta
>>>>>>>>>>>>>>>>>>> do, also when you pack your data into different
>>>>> formats,
>>>>>>>>>> usually
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
>>>>> data
>>>>>>> and
>>>>>>>>> we
>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>> really change that. It is more important for users to
>>>>>>>> prevent
>>>>>>>>>>>> useless
>>>>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
>>>>> to
>>>>>>> drop
>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> are not going to change my join output.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> usually it's:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
>>>>>>>> old.equals(new)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
>>>>> for
>>>>>>>>>> smaller
>>>>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
>>>>> then
>>>>>>>>> anyways.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
>>>>>>> sure if
>>>>>>>>>> it's
>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
>>>>> the
>>>>>>>>>> original
>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>> as I
>>>>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
>>>>> keys in
>>>>>>>> the
>>>>>>>>>>> value:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>>>> 1: A,...
>>>>>>>>>>>>>>>>>>>> 2: B,...
>>>>>>>>>>>>>>>>>>>> 3: A,...
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Gets re-keyed:
>>>>>>>>>>>>>>>>>>>> A: [1, 3]
>>>>>>>>>>>>>>>>>>>> B: [2]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
>>>>>>> regular
>>>>>>>>>>>> single-key
>>>>>>>>>>>>>>>>>>> lookup.
>>>>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
>>>>>>>> values,
>>>>>>>>> as
>>>>>>>>>>>>>>>>> there's
>>>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
>>>>> reference
>>>>>>> rhs
>>>>>>>>>>> records.
>>>>>>>>>>>>>>>>>>> Offhand,
>>>>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
>>>>>>> past
>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> threshold, we
>>>>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
>>>>> cases,
>>>>>>> it
>>>>>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
>>>>> be
>>>>>>> one
>>>>>>>>> per
>>>>>>>>>>>> (max
>>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
>>>>>>> there's
>>>>>>>>> some
>>>>>>>>>>>> extra
>>>>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
>>>>>>>> retraction.
>>>>>>>>>> For
>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
>>>>> just
>>>>>>> have
>>>>>>>>> (A:
>>>>>>>>>>>> [3]).
>>>>>>>>>>>>>>>>>>> Some
>>>>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
>>>>>>> for 1
>>>>>>>>> can
>>>>>>>>>>> also
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> retracted.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> That's all!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
>>>>> thoughtful
>>>>>>>> KIP.
>>>>>>>>>>> Sorry
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> discussion has been slow.
>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
>>>>> up,
>>>>>>> it
>>>>>>>>> just
>>>>>>>>>>>> goes
>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>> to discuss.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> would not expect to much attention with another
>>>>>>> another
>>>>>>>>> email
>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> best Jan
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>> Hello Contributors
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
>>>>>>> need
>>>>>>>> to
>>>>>>>>>> bump
>>>>>>>>>>>>>>>>> this to
>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
>>>>>>> through
>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>> contributor
>>>>>>>>>>>>>>>>>>>>>> feedback is given.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Main points that need addressing:
>>>>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
>>>>>>>>> singular
>>>>>>>>>>>> graph
>>>>>>>>>>>>>>>>> node
>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
>>>>> right
>>>>>>>>> now. I
>>>>>>>>>>> am
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>> fuzzy
>>>>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
>>>>> I
>>>>>>>> would
>>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>> help on this aspect.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
>>>>>>> thread
>>>>>>>>> has
>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>> discourse
>>>>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
>>>>> mark
>>>>>>>>>> proposal
>>>>>>>>>>>> and
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
>>>>> need
>>>>>>> to
>>>>>>>>>>> strictly
>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
>>>>> up
>>>>>>> to
>>>>>>>> the
>>>>>>>>>>>>>>>>> consumer.
>>>>>>>>>>>>>>>>>>> Any
>>>>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
>>>>> this
>>>>>>>> from
>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>>>>> vote?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>>>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
>>>>>>>>>>>>>>>>> Stores.persistentWindowStore,
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
>>>>> segments
>>>>>>>> you
>>>>>>>>>> want
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>> large
>>>>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
>>>>>>> happens
>>>>>>>>> is
>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
>>>>> data
>>>>>>> with
>>>>>>>>> new
>>>>>>>>>>>>>>>>>>> timestamps
>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
>>>>> of
>>>>>>> the
>>>>>>>>>> work
>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
>>>>> the
>>>>>>>>>> hopping
>>>>>>>>>>>>>>>>> windowed
>>>>>>>>>>>>>>>>>>>>> store.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
>>>>>>> to be
>>>>>>>>> two
>>>>>>>>>>> 24h
>>>>>>>>>>>>>>>>>>>>> segments.
>>>>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
>>>>> will
>>>>>>> be
>>>>>>>>>>> inserted
>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>> T in
>>>>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
>>>>> overlap
>>>>>>> by
>>>>>>>>> 12h.
>>>>>>>>>>> As
>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>> goes on
>>>>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
>>>>> the
>>>>>>>>> oldest
>>>>>>>>>>>>>> segment
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
>>>>> The
>>>>>>>>>> records
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
>>>>> it
>>>>>>> is
>>>>>>>>> the
>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
>>>>> are
>>>>>>>>>> retained
>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
>>>>> TTL
>>>>>>>> at a
>>>>>>>>>>>>>>>>>>> segment-level,
>>>>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
>>>>>>>>>>>> ClassCastException
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>> similar.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
>>>>>>>>> fetch/put
>>>>>>>>>> or
>>>>>>>>>>>>>>>>> iterate
>>>>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
>>>>>>> stream-time
>>>>>>>>>> under
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> hoods
>>>>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
>>>>> all
>>>>>>> the
>>>>>>>>>>> windows
>>>>>>>>>>>>>>>>>>> earlier?
>>>>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
>>>>>>> time
>>>>>>>>>>>> gurantees.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Best
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
>>>>>>> changed
>>>>>>>>> the
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
>>>>>>>> something
>>>>>>>>>> that
>>>>>>>>>>>>>>>>> will be
>>>>>>>>>>>>>>>>>>>>>>>>>>> extremely
>>>>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
>>>>> said, I
>>>>>>>> will
>>>>>>>>>> not
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>>>>>>>> ahead
>>>>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
>>>>> reshuffle/groupBy
>>>>>>>>> solution
>>>>>>>>>>> as
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>> propose.
>>>>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
>>>>>>>> yourself
>>>>>>>>>> off
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>> current PR
>>>>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
>>>>>>> would
>>>>>>>>> be
>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>> happy to
>>>>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
>>>>>>>> stands
>>>>>>>>>>> right
>>>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>>>>> I do
>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
>>>>>>> alternatives
>>>>>>>>>> without
>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
>>>>> which
>>>>>>>> they
>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> prefer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
>>>>> gave
>>>>>>> up
>>>>>>>> on
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
>>>>> pulled
>>>>>>>> this
>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>>>>>>> already.
>>>>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
>>>>> based on
>>>>>>>>> PAPI.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
>>>>>>> windowed
>>>>>>>>>> state
>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
>>>>> then I
>>>>>>>> will
>>>>>>>>>> come
>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
>>>>> any of
>>>>>>>> the
>>>>>>>>>>>> binding
>>>>>>>>>>>>>>>>>>> voters
>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
>>>>>>> updated
>>>>>>>> it
>>>>>>>>>>>>>>>>> according
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
>>>>> windowed
>>>>>>>>> store.
>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
>>>>> they
>>>>>>>> are
>>>>>>>>>>>>>>>>> completed.
>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
>>>>>>>> already
>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> PR?
>>>>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
>>>>> Missing
>>>>>>>>>>> something?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
>>>>> Wang <
>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
>>>>> is
>>>>>>> the
>>>>>>>>>> wrong
>>>>>>>>>>>>>> link,
>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
>>>>>>> part of
>>>>>>>>>>> KIP-258
>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
>>>>>>>> KTable"
>>>>>>>>>> such
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
>>>>> materialized
>>>>>>>> store,
>>>>>>>>>>> i.e.
>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
>>>>> older
>>>>>>>> than
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>> key's
>>>>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
>>>>> ordering.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
>>>>>>> Wang <
>>>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
>>>>>>> final
>>>>>>>>> step
>>>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
>>>>>>> with
>>>>>>>> a
>>>>>>>>>>> window
>>>>>>>>>>>>>>>>>>> store),
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
>>>>>>> help:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
>>>>>>> key-value
>>>>>>>>>> store
>>>>>>>>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
>>>>>>> usage,
>>>>>>>> as
>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
>>>>>>>> ,
>>>>>>>>> is
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
>>>>> its
>>>>>>>>>>> timestamp
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> smaller
>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
>>>>> I
>>>>>>>> think
>>>>>>>>> it
>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
>>>>> based
>>>>>>>>>>> filtering,
>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
>>>>> the
>>>>>>>>> joining
>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
>>>>>>> final
>>>>>>>>>> stage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
>>>>> and
>>>>>>>> hence
>>>>>>>>>>>>>>>>>>> non-windowed
>>>>>>>>>>>>>>>>>>>>>>>> KTables
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
>>>>>>> really
>>>>>>>>> have
>>>>>>>>>> a
>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
>>>>>>>>> KTable-KTable
>>>>>>>>>>>>>>>>> non-key
>>>>>>>>>>>>>>>>>>>>> joins
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
>>>>> Filipiak
>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
>>>>> would
>>>>>>>> grow
>>>>>>>>>>>>>> endlessly
>>>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
>>>>> pair
>>>>>>> of
>>>>>>>>>> (<this
>>>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
>>>>> is
>>>>>>> used
>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> differentiate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
>>>>>>> proposal
>>>>>>>>>> would
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> replace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
>>>>>>> This
>>>>>>>>> would
>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
>>>>> time.
>>>>>>> This
>>>>>>>>>>> should
>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
>>>>>>>> should
>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> customizable
>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
>>>>>>> perhaps
>>>>>>>>> just
>>>>>>>>>>> 10
>>>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
>>>>> do
>>>>>>> the
>>>>>>>>>> trick
>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>> Even
>>>>>>>>>>>>>>>>>>>>>>>> if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
>>>>>>>>> repartitioning
>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
>>>>>>> am a
>>>>>>>>>> little
>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>> sceptical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
>>>>> one
>>>>>>>>> could
>>>>>>>>>>> run
>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
>>>>>>> border. I
>>>>>>>>> will
>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
>>>>>>> problematic, we
>>>>>>>>>> could
>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
>>>>> performance
>>>>>>>>> impact I
>>>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
>>>>> correct
>>>>>>> as
>>>>>>>>> is. I
>>>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
>>>>>>>>> timestamp(A)  <
>>>>>>>>>>>>>>>>>>>>> timestamp(B).
>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
>>>>> -
>>>>>>>> thanks
>>>>>>>>>> for
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> diagram, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
>>>>> do
>>>>>>> not
>>>>>>>>> have
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
>>>>>>>>> available
>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
>>>>> Map.
>>>>>>>> That
>>>>>>>>>>> being
>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
>>>>> for
>>>>>>>>> clarity
>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> everyone
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
>>>>> really
>>>>>>> hard
>>>>>>>>>> work.
>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
>>>>>>>>> original
>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
>>>>> PAPI
>>>>>>>> and
>>>>>>>>>>>>>> basically
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
>>>>> missed
>>>>>>>> that
>>>>>>>>> in
>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
>>>>>>> up on
>>>>>>>>> my
>>>>>>>>>>> end.
>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>>>> finish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
>>>>>>> week.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
>>>>> the
>>>>>>> Map
>>>>>>>>> stay
>>>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> State
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
>>>>> changes
>>>>>>>> have
>>>>>>>>>>>>>>>>> propagated?
>>>>>>>>>>>>>>>>>>>>> Isn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
>>>>>>> state
>>>>>>>>>> store?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
>>>>>>> substractor
>>>>>>>> is
>>>>>>>>>>> gonna
>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>> `null`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
>>>>>>> there
>>>>>>>> is
>>>>>>>>>>> going
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
>>>>> this
>>>>>>>> store
>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
>>>>> is a
>>>>>>>>>> regular
>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>> satisfying
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
>>>>>>> join.
>>>>>>>>> The
>>>>>>>>>>>>>>>>> Windowed
>>>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
>>>>> statefull
>>>>>>>>>> operation
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
>>>>>>> have
>>>>>>>> the
>>>>>>>>>>>> window
>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
>>>>> custom
>>>>>>>> group
>>>>>>>>>> by
>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
>>>>> key i
>>>>>>>> think
>>>>>>>>>> it
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
>>>>> the
>>>>>>>>>> original
>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
>>>>>>> foundation
>>>>>>>>> to
>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
>>>>> user.
>>>>>>>> very
>>>>>>>>>>>>>>>>>>> unfortunate. I
>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
>>>>> do
>>>>>>> not
>>>>>>>>>> think
>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
>>>>> Prajakta
>>>>>>>>> Dumbre <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please remove me from this
>>>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             On Tue, Sep 11, 2018 at 1:29 PM
>>>>>>> Jan
>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hi Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > give me some time, will make
>>>>>>> such a
>>>>>>>>>>> chart.
>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>> time i
>>>>>>>>>>>>>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             get along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > well with giphy and ruined
>>>>> all
>>>>>>> your
>>>>>>>>>>> charts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hopefully i can get it done
>>>>>>> today
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > On 08.09.2018 16:00, Adam
>>>>>>> Bellemare
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I have included a diagram
>>>>> of
>>>>>>>> what I
>>>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > I attempted this back at
>>>>> the
>>>>>>>> start
>>>>>>>>> of
>>>>>>>>>>> my
>>>>>>>>>>>> own
>>>>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > solution, and since I could
>>>>>>> not
>>>>>>>> get
>>>>>>>>>> it
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> work I
>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             discarded the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > code. At this point in
>>>>> time,
>>>>>>> if
>>>>>>>> you
>>>>>>>>>>> wish
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             for your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > groupBy solution, I ask
>>>>> that
>>>>>>> you
>>>>>>>>>> please
>>>>>>>>>>>>>>>>> create a
>>>>>>>>>>>>>>>>>>>>>>>> diagram on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > carefully explaining your
>>>>>>>> solution.
>>>>>>>>>>>> Please
>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>>>> free
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the image I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > just posted as a starting
>>>>>>> point.
>>>>>>>> I
>>>>>>>>> am
>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>>>> trouble
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             understanding your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > explanations but I think
>>>>> that
>>>>>>> a
>>>>>>>>>>> carefully
>>>>>>>>>>>>>>>>>>> constructed
>>>>>>>>>>>>>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             will clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > any misunderstandings.
>>>>>>>> Alternately,
>>>>>>>>>>>> please
>>>>>>>>>>>>>>>>> post a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             comprehensive PR with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > your solution. I can only
>>>>>>> guess
>>>>>>>> at
>>>>>>>>>> what
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> mean, and
>>>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             value my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > own
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > time as much as you value
>>>>>>> yours,
>>>>>>>> I
>>>>>>>>>>>> believe
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             responsibility to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > provide an implementation
>>>>>>> instead
>>>>>>>>> of
>>>>>>>>>> me
>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > On Sat, Sep 8, 2018 at 8:00
>>>>>>> AM,
>>>>>>>> Jan
>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > > wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Hi James,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> nice to see you beeing
>>>>>>>> interested.
>>>>>>>>>>> kafka
>>>>>>>>>>>>>>>>>>> streams at
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             point supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> all sorts of joins as
>>>>> long as
>>>>>>>> both
>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Adam is currently
>>>>>>> implementing a
>>>>>>>>>> join
>>>>>>>>>>>>>> where a
>>>>>>>>>>>>>>>>>>> KTable
>>>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             KTable can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> a one to many relation
>>>>> ship
>>>>>>>> (1:n).
>>>>>>>>>> We
>>>>>>>>>>>>>> exploit
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> datastore that keeps data
>>>>>>> sorted
>>>>>>>> (At
>>>>>>>>>>> least
>>>>>>>>>>>>>>>>>>> exposes an
>>>>>>>>>>>>>>>>>>>>>>>> API to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> stored data in a sorted
>>>>>>>> fashion).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I think the technical
>>>>> caveats
>>>>>>>> are
>>>>>>>>>> well
>>>>>>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > basically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> down to philosophy and API
>>>>>>>> Design
>>>>>>>>> (
>>>>>>>>>>> when
>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>> sees
>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>> newest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             message).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> I have a lengthy track
>>>>>>> record of
>>>>>>>>>>> loosing
>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>> kinda
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arguments within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> streams community and I
>>>>> have
>>>>>>> no
>>>>>>>>> clue
>>>>>>>>>>>> why.
>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> literally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             can't wait for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> to churn through this
>>>>> thread
>>>>>>> and
>>>>>>>>>> give
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> opinion on
>>>>>>>>>>>>>>>>>>>>>>>> how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the return type of the
>>>>>>>>> oneToManyJoin
>>>>>>>>>>> and
>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>> power we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             want to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> the user vs "simplicity"
>>>>>>> (where
>>>>>>>>>>>> simplicity
>>>>>>>>>>>>>>>>> isn't
>>>>>>>>>>>>>>>>>>>>>>>> really that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> need to understand it I
>>>>>>> argue)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> waiting for you to join
>>>>> in on
>>>>>>>> the
>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> Best Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >> On 07.09.2018 15:49, James
>>>>>>> Kwan
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I am new to this group
>>>>> and I
>>>>>>>>> found
>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> subject
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             interesting.  Sounds
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> you guys want to
>>>>> implement a
>>>>>>>> join
>>>>>>>>>>>> table of
>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>> streams? Is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > somewhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> I can see the original
>>>>>>>>> requirement
>>>>>>>>>> or
>>>>>>>>>>>>>>>>> proposal?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>> On Sep 7, 2018, at 8:13
>>>>> AM,
>>>>>>> Jan
>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <Jan.Filipiak@trivago.com
>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> On 05.09.2018 22:17,
>>>>> Adam
>>>>>>>>>> Bellemare
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> I'm currently testing
>>>>>>> using a
>>>>>>>>>>>> Windowed
>>>>>>>>>>>>>>>>> Store
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> store the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             highwater
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> mark.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> By all indications this
>>>>>>>> should
>>>>>>>>>> work
>>>>>>>>>>>>>> fine,
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> caveat
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             being that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> only resolve
>>>>> out-of-order
>>>>>>>>> arrival
>>>>>>>>>>>> for up
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> size of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the window
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > (ie:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> 24h, 72h, etc). This
>>>>> would
>>>>>>>>> remove
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> possibility
>>>>>>>>>>>>>>>>>>>>>>>> of it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > unbounded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> size.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> With regards to Jan's
>>>>>>>>>> suggestion, I
>>>>>>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> remain in disagreement.
>>>>>>>> While I
>>>>>>>>>> do
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> disagree
>>>>>>>>>>>>>>>>>>>>>>>> with your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             statement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> there likely to be
>>>>>>> additional
>>>>>>>>>> joins
>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             workflow, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> see how you can
>>>>>>> conclusively
>>>>>>>>> deal
>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> foreign-key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> changes and subsequent
>>>>>>>> joins. I
>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             think you have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> proposed (without a
>>>>>>>> high-water,
>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>> groupBy and
>>>>>>>>>>>>>>>>>>>>>>>> reduce)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             and found
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> that if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> the foreign key changes
>>>>>>> too
>>>>>>>>>>> quickly,
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> load
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             stream thread
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> too
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> high, the joined
>>>>> messages
>>>>>>>> will
>>>>>>>>>>> arrive
>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>> and be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             incorrectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> propagated, such that
>>>>> an
>>>>>>>>>>> intermediate
>>>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> event.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Can you shed some light
>>>>> on
>>>>>>>> your
>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             There must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> some sort of flaw in it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I have a suspicion
>>>>> where it
>>>>>>>> is,
>>>>>>>>> I
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             confirm. The idea
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> is bullet proof and it
>>>>>>> must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> an implementation mess
>>>>> up.
>>>>>>> I
>>>>>>>>> would
>>>>>>>>>>>> like
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> clarify
>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we draw a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> conclusion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>    Repartitioning the
>>>>>>>> scattered
>>>>>>>>>>> events
>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> partitions is the only
>>>>> way I
>>>>>>>> know
>>>>>>>>>> how
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> conclusively
>>>>>>>>>>>>>>>>>>>>>>>> deal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> out-of-order events in
>>>>> a
>>>>>>>> given
>>>>>>>>>> time
>>>>>>>>>>>>>> frame,
>>>>>>>>>>>>>>>>>>> and to
>>>>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> eventually consistent
>>>>> with
>>>>>>>> the
>>>>>>>>>>> input
>>>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> If you have some code
>>>>> to
>>>>>>>> share
>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> illustrates
>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             approach, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> very grateful as it
>>>>> would
>>>>>>>>> remove
>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>> misunderstandings
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that I may
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> ah okay you were looking
>>>>>>> for
>>>>>>>> my
>>>>>>>>>>> code.
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             something easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> readable here as its
>>>>>>> bloated
>>>>>>>>> with
>>>>>>>>>>>>>>>>> OO-patterns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> its anyhow trivial:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> @Override
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      public T apply(K
>>>>>>> aggKey,
>>>>>>>> V
>>>>>>>>>>>> value, T
>>>>>>>>>>>>>>>>>>>>> aggregate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          Map<U, V>
>>>>>>>>>>> currentStateAsMap =
>>>>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             imaginary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          U toModifyKey =
>>>>>>>>>>>>>>>>> mapper.apply(value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << this is
>>>>> the
>>>>>>>>> place
>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             gonna have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and why you probably
>>>>>>> couldn't
>>>>>>>> do
>>>>>>>>>> it.
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>> to find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             a solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> I didn't realize that
>>>>> yet.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << we
>>>>>>> propagate
>>>>>>>> the
>>>>>>>>>>>> field in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> joiner, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             that we can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > pick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> it up in an aggregate.
>>>>>>>> Probably
>>>>>>>>>> you
>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> thought
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             this in your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> approach right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I am
>>>>> very
>>>>>>> open
>>>>>>>>> to
>>>>>>>>>>>> find a
>>>>>>>>>>>>>>>>>>> generic
>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             here. In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> honest opinion this is
>>>>>>> broken
>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             looses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> and only maintains the
>>>>>>>> aggregate
>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              << I
>>>>>>> abstracted
>>>>>>>> it
>>>>>>>>>> away
>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>> then way
>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> of oneToMany join. That
>>>>> is
>>>>>>>> why I
>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>> realize
>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             significance here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              <<
>>>>> Opinions?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          for (V m :
>>>>>>> current)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>> currentStateAsMap.put(mapper.apply(m),
>>>>>>>>>>>>>> m);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          if (isAdder)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>> currentStateAsMap.put(toModifyKey,
>>>>>>>>>>>>>> value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          else
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>                  return
>>>>>>> null;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>          retrun
>>>>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>      }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> On Wed, Sep 5, 2018 at
>>>>>>> 3:35
>>>>>>>> PM,
>>>>>>>>>> Jan
>>>>>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Jan.Filipiak@trivago.com
>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>> Thanks Adam for
>>>>> bringing
>>>>>>>>> Matthias
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> speed!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> about the
>>>>> differences. I
>>>>>>>> think
>>>>>>>>>>>>>> re-keying
>>>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             optional at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> best.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I would say we return
>>>>> a
>>>>>>>>>>>> KScatteredTable
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> reshuffle()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>> KTable<originalKey,Joined>
>>>>>>>> to
>>>>>>>>>> make
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartitioning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I am also in a big
>>>>>>> favour of
>>>>>>>>>> doing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             processing using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by instead high water
>>>>>>> mark
>>>>>>>>>>> tracking.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Just because unbounded
>>>>>>>> growth
>>>>>>>>> is
>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>> scary
>>>>>>>>>>>>>>>>>>> + It
>>>>>>>>>>>>>>>>>>>>>>>> saves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             the header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> stuff.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> I think the
>>>>> abstraction
>>>>>>> of
>>>>>>>>>> always
>>>>>>>>>>>>>>>>>>> repartitioning
>>>>>>>>>>>>>>>>>>>>>>>> back is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             just not so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> strong. Like the work
>>>>> has
>>>>>>>> been
>>>>>>>>>>> done
>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             back and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> grouping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> by something else
>>>>>>> afterwards
>>>>>>>>> is
>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>> common.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> On 05.09.2018 13:49,
>>>>> Adam
>>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>> Hi Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thank you for your
>>>>>>>> feedback,
>>>>>>>>> I
>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> While name spacing
>>>>>>> would be
>>>>>>>>>>>> possible,
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
>>>>>>> implies
>>>>>>>> a
>>>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now to
>>>>>>> avoid
>>>>>>>>> the
>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
>>>>>>> still
>>>>>>>> add
>>>>>>>>>>> name
>>>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Agreed. I will go
>>>>> with
>>>>>>>>> using a
>>>>>>>>>>>>>> reserved
>>>>>>>>>>>>>>>>>>> string
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             document it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> My main concern about
>>>>>>> the
>>>>>>>>>> design
>>>>>>>>>>> it
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> type of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> understood the
>>>>> proposal
>>>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> In your example, you
>>>>>>> have
>>>>>>>>>> table1
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>>>> swapped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Here is how it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> works
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> currently:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 1) table1 has the
>>>>>>> records
>>>>>>>>> that
>>>>>>>>>>>> contain
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> foreign key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             within their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> value.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 input stream:
>>>>>>>>>>>> <a,(fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table2 input stream:
>>>>>>> <A,X>,
>>>>>>>>>> <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 2) A Value mapper is
>>>>>>>> required
>>>>>>>>>> to
>>>>>>>>>>>>>> extract
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 foreign key
>>>>>>> mapper:
>>>>>>>> (
>>>>>>>>>>> value
>>>>>>>>>>>> =>
>>>>>>>>>>>>>>>>>>> value.fk
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <http://value.fk> )
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> The mapper is
>>>>> applied to
>>>>>>>> each
>>>>>>>>>>>> element
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             new combined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> key is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> made:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> table1 mapped: <A-a,
>>>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             <B-c,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 3) The rekeyed events
>>>>>>> are
>>>>>>>>>>>>>> copartitioned
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> table2:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> a) Stream Thread with
>>>>>>>>> Partition
>>>>>>>>>>> 0:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> RepartitionedTable1:
>>>>>>> <A-a,
>>>>>>>>>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             (fk=A,bar=2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> b) Stream Thread with
>>>>>>>>> Partition
>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> RepartitionedTable1:
>>>>>>> <B-c,
>>>>>>>>>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> 4) From here, they
>>>>> can
>>>>>>> be
>>>>>>>>>> joined
>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             applying the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> joiner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> At this point, Jan's
>>>>>>> design
>>>>>>>>> and
>>>>>>>>>>> my
>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>> deviate. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design goes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> repartition the data
>>>>>>>>> post-join
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> resolve
>>>>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             arrival of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> records,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> finally returning the
>>>>>>> data
>>>>>>>>>> keyed
>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> original key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             I do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> expose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> CombinedKey or any of
>>>>>>> the
>>>>>>>>>>> internals
>>>>>>>>>>>>>>>>>>> outside of
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             joinOnForeignKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> function. This does
>>>>> make
>>>>>>>> for
>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>> footprint,
>>>>>>>>>>>>>>>>>>>>>>>> but it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             removes all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> agency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> for resolving
>>>>>>> out-of-order
>>>>>>>>>>> arrivals
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             CombinedKeys from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> user. I believe that
>>>>>>> this
>>>>>>>>> makes
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> function
>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to use.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Let me know if this
>>>>>>> helps
>>>>>>>>>> resolve
>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>> questions,
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             please feel
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> add anything else on
>>>>>>> your
>>>>>>>>> mind.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> On Tue, Sep 4, 2018
>>>>> at
>>>>>>> 8:36
>>>>>>>>> PM,
>>>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>> matthias@confluent.io
>>>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I am just catching
>>>>> up
>>>>>>> on
>>>>>>>>> this
>>>>>>>>>>>>>> thread. I
>>>>>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             everything so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> far,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> but want to share
>>>>>>> couple
>>>>>>>> of
>>>>>>>>>>>> initial
>>>>>>>>>>>>>>>>>>> thoughts:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Headers: I think
>>>>> there
>>>>>>> is
>>>>>>>> a
>>>>>>>>>>>>>> fundamental
>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             between header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> usage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> in this KIP and
>>>>> KP-258.
>>>>>>>> For
>>>>>>>>>> 258,
>>>>>>>>>>>> we
>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are owned by Kafka
>>>>>>> Streams
>>>>>>>>> and
>>>>>>>>>>>> nobody
>>>>>>>>>>>>>>>>>>> else is
>>>>>>>>>>>>>>>>>>>>>>>> supposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             to write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> them. In fact, no
>>>>> user
>>>>>>>>> header
>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> thus, there are not
>>>>>>>>> conflicts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Nevertheless, I
>>>>> don't
>>>>>>> see
>>>>>>>> a
>>>>>>>>>> big
>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headers within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> As long as we
>>>>> document
>>>>>>> it,
>>>>>>>>> we
>>>>>>>>>>> can
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>> "reserved"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             header keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> users are not
>>>>> allowed
>>>>>>> to
>>>>>>>> use
>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>> data with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this should be
>>>>>>> ok.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> I think there is a
>>>>> safe
>>>>>>>> way
>>>>>>>>> to
>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>> conflicts,
>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > headers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> only needed in
>>>>>>> internal
>>>>>>>>>> topics
>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>> think):
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> For internal and
>>>>>>>> changelog
>>>>>>>>>>>> topics,
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>> namespace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             all headers:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * user-defined
>>>>> headers
>>>>>>>> are
>>>>>>>>>>>>>> namespaced
>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>> "external."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> * internal headers
>>>>> are
>>>>>>>>>>>> namespaced as
>>>>>>>>>>>>>>>>>>>>>>>> "internal." +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> While name spacing
>>>>>>> would
>>>>>>>> be
>>>>>>>>>>>>>> possible,
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> user headers what
>>>>>>> implies
>>>>>>>> a
>>>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> namespace for now to
>>>>>>> avoid
>>>>>>>>> the
>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the future, we can
>>>>>>> still
>>>>>>>> add
>>>>>>>>>>> name
>>>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> My main concern
>>>>> about
>>>>>>> the
>>>>>>>>>> design
>>>>>>>>>>>> it
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> If I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> understood the
>>>>> proposal
>>>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V1>
>>>>> table1 =
>>>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K2,V2>
>>>>> table2 =
>>>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> KTable<K1,V3>
>>>>>>> joinedTable
>>>>>>>> =
>>>>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> implies that the
>>>>>>>>> `joinedTable`
>>>>>>>>>>> has
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             left input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> IMHO, this does not
>>>>>>> work
>>>>>>>>>> because
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>>>> contains
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             multiple rows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join with a record
>>>>> in
>>>>>>>> table1
>>>>>>>>>>>> (what is
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>>>> purpose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> join), the result
>>>>> table
>>>>>>>>> would
>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>> contain a
>>>>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             join result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> multiple.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Example:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table1 input stream:
>>>>>>> <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> table2 input stream:
>>>>>>>>>> <a,(A,1)>,
>>>>>>>>>>>>>>>>> <b,(A,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> We use table2 value
>>>>> a
>>>>>>>>> foreign
>>>>>>>>>>> key
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> table1
>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             "A" joins).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result key is the
>>>>> same
>>>>>>> key
>>>>>>>>> as
>>>>>>>>>>> key
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             implies that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> result can either be
>>>>>>> <A,
>>>>>>>>>>>> join(X,1)>
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> <A,
>>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > both.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Because the share
>>>>> the
>>>>>>> same
>>>>>>>>>> key,
>>>>>>>>>>>>>>>>> whatever
>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             we emit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> overwrite the
>>>>> previous
>>>>>>>>> result.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This is the reason
>>>>> why
>>>>>>> Jan
>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > combination
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> both primary keys of
>>>>>>> the
>>>>>>>>> input
>>>>>>>>>>>> tables
>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             output table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> makes the keys of
>>>>> the
>>>>>>>> output
>>>>>>>>>>> table
>>>>>>>>>>>>>>>>> unique
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             store both in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> output table:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Result would be
>>>>> <A-a,
>>>>>>>>>>> join(X,1)>,
>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> On 9/4/18 1:36 PM,
>>>>> Jan
>>>>>>>>>> Filipiak
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>> Just on remark here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> The high-watermark
>>>>>>> could
>>>>>>>> be
>>>>>>>>>>>>>>>>> disregarded.
>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> depends on the
>>>>> size of
>>>>>>>> the
>>>>>>>>>>>>>> aggregated
>>>>>>>>>>>>>>>>>>> map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> Only 1 element long
>>>>>>> maps
>>>>>>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> unpacked
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             forwarded. 0
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > element
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> maps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> would be published
>>>>> as
>>>>>>>>> delete.
>>>>>>>>>>> Any
>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>> count
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of map entries is
>>>>> in
>>>>>>>>> "waiting
>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> correct
>>>>>>>>>>>>>>>>>>>>>>>> deletes to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > arrive"-state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> On 04.09.2018
>>>>> 21:29,
>>>>>>> Adam
>>>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> It does look like I
>>>>>>> could
>>>>>>>>>>> replace
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             repartition store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> highwater store
>>>>> with
>>>>>>> a
>>>>>>>>>> groupBy
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> reduce.
>>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             it looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> still need to
>>>>> store
>>>>>>> the
>>>>>>>>>>>> highwater
>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>>>> within
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             materialized
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> compare the
>>>>> arrival of
>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>>> (assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> THIS is
>>>>> correct...).
>>>>>>> This
>>>>>>>>> in
>>>>>>>>>>>> effect
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             design I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>> just with the two
>>>>>>> tables
>>>>>>>>>> merged
>>>>>>>>>>>>>>>>> together.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> -- Guozhang
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Jan

I have been thinking a lot about the history of the discussion and your
original proposal, and why you believe it is a better solution. The biggest
problem with your original proposed design is that it seems to me to be
non-deterministic. It is subject to race conditions that are dependent
entirely on the data, and without resolution of these races you can end up
with different results each time. If I am mistaken and this is indeed
deterministic, then please let me know and provide an explanation, ideally
with an example.

The way I see it is that you will get very different answers to your
non-race-condition-resolved join topology, especially if you are nesting it
with additional joins as you have indicated you are doing. Consider
rebuilding an application state from the beginning of two topics. If the
left/this side has multiple foreign-key changes in a row, spaced out every
ten minutes, you may see something like this:

(foo, foreignKey=red) t=0
(foo, foreignKey=blue) t=0+10m
(foo, foreignKey=green) t=0+20m
(foo, foreignKey=purple) t=0+30m
(foo, foreignKey=blue) t=0+40m
(foo, foreignKey=white) t=0+50m

During realtime processing, all of the updates may have correctly
propagated because it took less than 10 minutes to resolve each join. Upon
rebuilding from the start, however, all of these events would be processed
in quick succession. The presence or absence of data will affect the
results of your join, and the results can vary with each run depending on
the data. Because of this, I cannot support any kind of solution that would
allow the exposure of an unresolved intermediate state. I can understand if
you don't support this, but this is why, as you said, you have the freedom
to use the Processor API.


With that being said, either the solution that I originally proposed
(join's ocurring on the foreign node) or John + Guozhang's solution
(registering with the foreign node for notifications) is fine with me -
both have the same API and we can evaluate it further during implementation.


Thanks

Adam

On Thu, Dec 27, 2018 at 2:38 PM Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi,
>
> just want to let you guys know that this thing is spiralling out of
> control if you ask me.
>
> First you take away the possibility for the user to optimize. Now you
> pile up complexity to perform some afterwards optimisation, that from my
> POV completely misses the point. As if the actual call to the joiner
> really gonna be an expensive part. It wont. Truth is, you don't have a
> clue which side is gonna be smaller. might be the key you shuffle around
> is >>> than the value on the other side already.
>
> You know my opinion on this. For me its dead, I just leave you the
> message here as an opportunity to reconsider the choices that were made.
>
> Whish y'll a happy new year :)
>
>
>
>
>
>
> On 27.12.2018 17:22, Adam Bellemare wrote:
> > Hi All
> >
> > Sorry for the delay - holidays and all. I have since updated the KIP with
> > John's original suggestion and have pruned a number of the no longer
> > relevant diagrams. Any more comments would be welcomed, otherwise I will
> > look to kick off the vote again shortly.
> >
> > Thanks
> > Adam
> >
> > On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <adam.bellemare@gmail.com
> >
> > wrote:
> >
> >> Hi John and Guozhang
> >>
> >> Ah yes, I lost that in the mix! Thanks for the convergent solutions - I
> do
> >> think that the attachment that John included makes for a better design.
> It
> >> should also help with overall performance as very high-cardinality
> foreign
> >> keyed data (say millions of events with the same entity) will be able to
> >> leverage the multiple nodes for join functionality instead of having it
> all
> >> performed in one node. There is still a bottleneck in the right table
> >> having to propagate all those events, but with slimmer structures, less
> IO
> >> and no need to perform the join I think the throughput will be much
> higher
> >> in those scenarios.
> >>
> >> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
> >> version of John's diagram and ensure that the example flow matches
> >> correctly. Then I can go back to working on the PR to match the diagram.
> >>
> >> Thanks both of you for all the help - very much appreciated.
> >>
> >> Adam
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> Hi John,
> >>>
> >>> Just made a pass on your diagram (nice hand-drawing btw!), and
> obviously
> >>> we
> >>> are thinking about the same thing :) A neat difference that I like, is
> >>> that
> >>> in the pre-join repartition topic we can still send message in the
> format
> >>> of `K=k, V=(i=2)` while using "i" as the partition key in
> >>> StreamsPartition,
> >>> this way we do not need to even augment the key for the repartition
> topic,
> >>> but just do a projection on the foreign key part but trim all other
> >>> fields:
> >>> as long as we still materialize the store as `A-2` co-located with the
> >>> right KTable, that is fine.
> >>>
> >>> As I mentioned in my previous email, I also think this has a few
> >>> advantages
> >>> on saving over-the-wire bytes as well as disk bytes.
> >>>
> >>> Guozhang
> >>>
> >>>
> >>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io>
> wrote:
> >>>
> >>>> Hi Guozhang,
> >>>>
> >>>> Thanks for taking a look! I think Adam's already addressed your
> >>> questions
> >>>> as well as I could have.
> >>>>
> >>>> Hi Adam,
> >>>>
> >>>> Thanks for updating the KIP. It looks great, especially how all the
> >>>> need-to-know information is right at the top, followed by the details.
> >>>>
> >>>> Also, thanks for that high-level diagram. Actually, now that I'm
> looking
> >>>> at it, I think part of my proposal got lost in translation, although I
> >>> do
> >>>> think that what you have there is also correct.
> >>>>
> >>>> I sketched up a crude diagram based on yours and attached it to the
> KIP
> >>>> (I'm not sure if attached or inline images work on the mailing list):
> >>>>
> >>>
> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
> >>>> . It's also attached to this email for convenience.
> >>>>
> >>>> Hopefully, you can see how it's intended to line up, and which parts
> are
> >>>> modified.
> >>>> At a high level, instead of performing the join on the right-hand
> side,
> >>>> we're essentially just registering interest, like "LHS key A wishes to
> >>>> receive updates for RHS key 2". Then, when there is a new "interest"
> or
> >>> any
> >>>> updates to the RHS records, it "broadcasts" its state back to the LHS
> >>>> records who are interested in it.
> >>>>
> >>>> Thus, instead of sending the LHS values to the RHS joiner workers and
> >>> then
> >>>> sending the join results back to the LHS worke be co-partitioned and
> >>>> validated, we instead only send the LHS *keys* to the RHS workers and
> >>> then
> >>>> only the RHS k/v back to be joined by the LHS worker.
> >>>>
> >>>> I've been considering both your diagram and mine, and I *think* what
> I'm
> >>>> proposing has a few advantages.
> >>>>
> >>>> Here are some points of interest as you look at the diagram:
> >>>> * When we extract the foreign key and send it to the Pre-Join
> >>> Repartition
> >>>> Topic, we can send only the FK/PK pair. There's no need to worry about
> >>>> custom partitioner logic, since we can just use the foreign key
> plainly
> >>> as
> >>>> the repartition record key. Also, we save on transmitting the LHS
> value,
> >>>> since we only send its key in this step.
> >>>> * We also only need to store the RHSKey:LHSKey mapping in the
> >>>> MaterializedSubscriptionStore, saving on disk. We can use the same
> rocks
> >>>> key format you proposed and the same algorithm involving range scans
> >>> when
> >>>> the RHS records get updated.
> >>>> * Instead of joining on the right side, all we do is compose a
> >>>> re-repartition record so we can broadcast the RHS k/v pair back to the
> >>>> original LHS partition. (this is what the "rekey" node is doing)
> >>>> * Then, there is a special kind of Joiner that's co-resident in the
> same
> >>>> StreamTask as the LHS table, subscribed to the Post-Join Repartition
> >>> Topic.
> >>>> ** This Joiner is *not* triggered directly by any changes in the LHS
> >>>> KTable. Instead, LHS events indirectly trigger the join via the whole
> >>>> lifecycle.
> >>>> ** For each event arriving from the Post-Join Repartition Topic, the
> >>>> Joiner looks up the corresponding record in the LHS KTable. It
> validates
> >>>> the FK as you noted, discarding any inconsistent events. Otherwise, it
> >>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the
> join
> >>>> result
> >>>> ** Note that the Joiner itself is stateless, so materializing the join
> >>>> result is optional, just as with the 1:1 joins.
> >>>>
> >>>> So in summary:
> >>>> * instead of transmitting the LHS keys and values to the right and the
> >>>> JoinResult back to the left, we only transmit the LHS keys to the
> right
> >>> and
> >>>> the RHS values to the left. Assuming the average RHS value is on
> smaller
> >>>> than or equal to the average join result size, it's a clear win on
> >>> broker
> >>>> traffic. I think this is actually a reasonable assumption, which we
> can
> >>>> discuss more if you're suspicious.
> >>>> * we only need one copy of the data (the left and right tables need to
> >>> be
> >>>> materialized) and one extra copy of the PK:FK pairs in the
> Materialized
> >>>> Subscription Store. Materializing the join result is optional, just as
> >>> with
> >>>> the existing 1:1 joins.
> >>>> * we still need the fancy range-scan algorithm on the right to locate
> >>> all
> >>>> interested LHS keys when a RHS value is updated, but we don't need a
> >>> custom
> >>>> partitioner for either repartition topic (this is of course a
> >>> modification
> >>>> we could make to your version as well)
> >>>>
> >>>> How does this sound to you? (And did I miss anything?)
> >>>> -John
> >>>>
> >>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
> >>> adam.bellemare@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> Hi John & Guozhang
> >>>>>
> >>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the
> >>> KIP,
> >>>>> pruned much of what I wrote and put a simplified diagram near the top
> >>> to
> >>>>> illustrate the workflow. I encapsulated Jan's content at the bottom
> of
> >>> the
> >>>>> document. I believe it is simpler to read by far now.
> >>>>>
> >>>>> @Guozhang Wang <wa...@gmail.com>:
> >>>>>> #1: rekey left table
> >>>>>>    -> source from the left upstream, send to rekey-processor to
> >>> generate
> >>>>> combined key, and then sink to copartition topic.
> >>>>> Correct.
> >>>>>
> >>>>>> #2: first-join with right table
> >>>>>>    -> source from the right table upstream, materialize the right
> >>> table.
> >>>>>>    -> source from the co-partition topic, materialize the rekeyed
> left
> >>>>> table, join with the right table, rekey back, and then sink to the
> >>>>> rekeyed-back topic.
> >>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I need
> the
> >>>>> Foreign-Key value generated in #1 above to compare in the resolution
> >>>>> stage.
> >>>>>
> >>>>>> #3: second join
> >>>>>>     -> source from the rekeyed-back topic, materialize the rekeyed
> >>> back
> >>>>> table.
> >>>>>>    -> source from the left upstream, materialize the left table,
> join
> >>>>> with
> >>>>> the rekeyed back table.
> >>>>> Almost - As each event comes in, we just run it through a stateful
> >>>>> processor that checks the original ("This") KTable for the key. The
> >>> value
> >>>>> payload then has the foreignKeyExtractor applied again as in Part #1
> >>>>> above,
> >>>>> and gets the current foreign key. Then we compare it to the joined
> >>> event
> >>>>> that we are currently resolving. If they have the same foreign-key,
> >>>>> propagate the result out. If they don't, throw the event away.
> >>>>>
> >>>>> The end result is that we do need to materialize 2 additional tables
> >>>>> (left/this-combinedkey table, and the final Joined table) as I've
> >>>>> illustrated in the updated KIP. I hope the diagram clears it up a lot
> >>>>> better. Please let me know.
> >>>>>
> >>>>> Thanks again
> >>>>> Adam
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
> >>> wrote:
> >>>>>
> >>>>>> John,
> >>>>>>
> >>>>>> Thanks a lot for the suggestions on refactoring the wiki, I agree
> >>> with
> >>>>> you
> >>>>>> that we should consider the KIP proposal to be easily understood by
> >>>>> anyone
> >>>>>> in the future to read, and hence should provide a good summary on
> the
> >>>>>> user-facing interfaces, as well as rejected alternatives to
> represent
> >>>>>> briefly "how we came a long way to this conclusion, and what we have
> >>>>>> argued, disagreed, and agreed about, etc" so that readers do not
> >>> need to
> >>>>>> dig into the DISCUSS thread to get all the details. We can, of
> >>> course,
> >>>>> keep
> >>>>>> the implementation details like "workflows" on the wiki page as a
> >>>>> addendum
> >>>>>> section since it also has correlations.
> >>>>>>
> >>>>>> Regarding your proposal on comment 6): that's a very interesting
> >>> idea!
> >>>>> Just
> >>>>>> to clarify that I understands it fully correctly: the proposal's
> >>>>> resulted
> >>>>>> topology is still the same as the current proposal, where we will
> >>> have 3
> >>>>>> sub-topologies for this operator:
> >>>>>>
> >>>>>> #1: rekey left table
> >>>>>>     -> source from the left upstream, send to rekey-processor to
> >>> generate
> >>>>>> combined key, and then sink to copartition topic.
> >>>>>>
> >>>>>> #2: first-join with right table
> >>>>>>     -> source from the right table upstream, materialize the right
> >>> table.
> >>>>>>     -> source from the co-partition topic, materialize the rekeyed
> >>> left
> >>>>>> table, join with the right table, rekey back, and then sink to the
> >>>>>> rekeyed-back topic.
> >>>>>>
> >>>>>> #3: second join
> >>>>>>     -> source from the rekeyed-back topic, materialize the rekeyed
> >>> back
> >>>>>> table.
> >>>>>>     -> source from the left upstream, materialize the left table,
> join
> >>>>> with
> >>>>>> the rekeyed back table.
> >>>>>>
> >>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology since
> >>>>> both of
> >>>>>> them read from the left table source stream. In this workflow, we
> >>> need
> >>>>> to
> >>>>>> materialize 4 tables (left table in #3, right table in #2, rekeyed
> >>> left
> >>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
> >>>>>> (copartition topic, rekeyed-back topic).
> >>>>>>
> >>>>>> Compared with Adam's current proposal in the workflow overview, it
> >>> has
> >>>>> the
> >>>>>> same num.materialize tables (left table, rekeyed left table, right
> >>>>> table,
> >>>>>> out-of-ordering resolver table), and same num.internal topics (two).
> >>> The
> >>>>>> advantage is that on the copartition topic, we can save bandwidth by
> >>> not
> >>>>>> sending value, and in #2 the rekeyed left table is smaller since we
> >>> do
> >>>>> not
> >>>>>> have any values to materialize. Is that right?
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
> >>> wrote:
> >>>>>>
> >>>>>>> Hi Adam,
> >>>>>>>
> >>>>>>> Given that the committers are all pretty busy right now, I think
> >>> that
> >>>>> it
> >>>>>>> would help if you were to refactor the KIP a little to reduce the
> >>>>>> workload
> >>>>>>> for reviewers.
> >>>>>>>
> >>>>>>> I'd recommend the following changes:
> >>>>>>> * relocate all internal details to a section at the end called
> >>>>> something
> >>>>>>> like "Implementation Notes" or something like that.
> >>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
> >>> mention
> >>>>>> only
> >>>>>>> publicly-facing API changes.
> >>>>>>> ** for example, the interface that you've already listed there, as
> >>>>> well
> >>>>>> as
> >>>>>>> a textual description of the guarantees we'll be providing (join
> >>>>> result
> >>>>>> is
> >>>>>>> copartitioned with the LHS, and the join result is guaranteed
> >>> correct)
> >>>>>>>
> >>>>>>> A good target would be that the whole main body of the KIP,
> >>> including
> >>>>>>> Status, Motivation, Proposal, Justification, and Rejected
> >>> Alternatives
> >>>>>> all
> >>>>>>> fit "above the fold" (i.e., all fit on the screen at a comfortable
> >>>>> zoom
> >>>>>>> level).
> >>>>>>> I think the only real Rejected Alternative that bears mention at
> >>> this
> >>>>>> point
> >>>>>>> is KScatteredTable, which you could just include the executive
> >>>>> summary on
> >>>>>>> (no implementation details), and link to extra details in the
> >>>>>>> Implementation Notes section.
> >>>>>>>
> >>>>>>> Taking a look at the wiki page, ~90% of the text there is internal
> >>>>>> detail,
> >>>>>>> which is useful for the dubious, but doesn't need to be ratified
> >>> in a
> >>>>>> vote
> >>>>>>> (and would be subject to change without notice in the future
> >>> anyway).
> >>>>>>> There's also a lot of conflicting discussion, as you've very
> >>>>> respectfully
> >>>>>>> tried to preserve the original proposal from Jan while adding your
> >>>>> own.
> >>>>>>> Isolating all this information in a dedicated section at the bottom
> >>>>> frees
> >>>>>>> the voters up to focus on the public API part of the proposal,
> >>> which
> >>>>> is
> >>>>>>> really all they need to consider.
> >>>>>>>
> >>>>>>> Plus, it'll be clear to future readers which parts of the document
> >>> are
> >>>>>>> enduring, and which parts are a snapshot of our implementation
> >>>>> thinking
> >>>>>> at
> >>>>>>> the time.
> >>>>>>>
> >>>>>>> I'm suggesting this because I suspect that the others haven't made
> >>>>> time
> >>>>>> to
> >>>>>>> review it partly because it seems daunting. If it seems like it
> >>> would
> >>>>> be
> >>>>>> a
> >>>>>>> huge time investment to review, people will just keep putting it
> >>> off.
> >>>>> But
> >>>>>>> if the KIP is a single page, then they'll be more inclined to give
> >>> it
> >>>>> a
> >>>>>>> read.
> >>>>>>>
> >>>>>>> Honestly, I don't think the KIP itself is that controversial (apart
> >>>>> from
> >>>>>>> the scattered table thing (sorry, Jan) ). Most of the discussion
> >>> has
> >>>>> been
> >>>>>>> around the implementation, which we can continue more effectively
> >>> in
> >>>>> a PR
> >>>>>>> once the KIP has passed.
> >>>>>>>
> >>>>>>> How does that sound?
> >>>>>>> -John
> >>>>>>>
> >>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
> >>>>> adam.bellemare@gmail.com
> >>>>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> 1) I believe that the resolution mechanism John has proposed is
> >>>>>>> sufficient
> >>>>>>>> - it is clean and easy and doesn't require additional RocksDB
> >>>>> stores,
> >>>>>>> which
> >>>>>>>> reduces the footprint greatly. I don't think we need to resolve
> >>>>> based
> >>>>>> on
> >>>>>>>> timestamp or offset anymore, but if we decide to do to that
> >>> would be
> >>>>>>> within
> >>>>>>>> the bounds of the existing API.
> >>>>>>>>
> >>>>>>>> 2) Is the current API sufficient, or does it need to be altered
> >>> to
> >>>>> go
> >>>>>>> back
> >>>>>>>> to vote?
> >>>>>>>>
> >>>>>>>> 3) KScatteredTable implementation can always be added in a future
> >>>>>>> revision.
> >>>>>>>> This API does not rule it out. This implementation of this
> >>> function
> >>>>>> would
> >>>>>>>> simply be replaced with `KScatteredTable.resolve()` while still
> >>>>>>> maintaining
> >>>>>>>> the existing API, thereby giving both features as Jan outlined
> >>>>> earlier.
> >>>>>>>> Would this work?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Thanks Guozhang, John and Jan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <john@confluent.io
> >>>>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi, all,
> >>>>>>>>>
> >>>>>>>>>>> In fact, we
> >>>>>>>>>>> can just keep a single final-result store with timestamps
> >>> and
> >>>>>> reject
> >>>>>>>>> values
> >>>>>>>>>>> that have a smaller timestamp, is that right?
> >>>>>>>>>
> >>>>>>>>>> Which is the correct output should at least be decided on the
> >>>>>> offset
> >>>>>>> of
> >>>>>>>>>> the original message.
> >>>>>>>>>
> >>>>>>>>> Thanks for this point, Jan.
> >>>>>>>>>
> >>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
> >>> the
> >>>>> k/v
> >>>>>>>>> store,
> >>>>>>>>> as well as providing a storage-format upgrade path.
> >>>>>>>>>
> >>>>>>>>> I might have missed it, but I think we have yet to discuss
> >>> whether
> >>>>>> it's
> >>>>>>>>> safe
> >>>>>>>>> or desirable just to swap topic-ordering our for
> >>>>> timestamp-ordering.
> >>>>>>> This
> >>>>>>>>> is
> >>>>>>>>> a very deep topic, and I think it would only pollute the
> >>> current
> >>>>>>>>> discussion.
> >>>>>>>>>
> >>>>>>>>> What Adam has proposed is safe, given the *current* ordering
> >>>>>> semantics
> >>>>>>>>> of the system. If we can agree on his proposal, I think we can
> >>>>> merge
> >>>>>>> the
> >>>>>>>>> feature well before the conversation about timestamp ordering
> >>> even
> >>>>>>> takes
> >>>>>>>>> place, much less reaches a conclusion. In the mean time, it
> >>> would
> >>>>>> seem
> >>>>>>> to
> >>>>>>>>> be unfortunate to have one join operator with different
> >>> ordering
> >>>>>>>> semantics
> >>>>>>>>> from every other KTable operator.
> >>>>>>>>>
> >>>>>>>>> If and when that timestamp discussion takes place, many (all?)
> >>>>> KTable
> >>>>>>>>> operations
> >>>>>>>>> will need to be updated, rendering the many:one join a small
> >>>>> marginal
> >>>>>>>> cost.
> >>>>>>>>>
> >>>>>>>>> And, just to plug it again, I proposed an algorithm above that
> >>> I
> >>>>>>> believe
> >>>>>>>>> provides
> >>>>>>>>> correct ordering without any additional metadata, and
> >>> regardless
> >>>>> of
> >>>>>> the
> >>>>>>>>> ordering semantics. I didn't bring it up further, because I
> >>> felt
> >>>>> the
> >>>>>>> KIP
> >>>>>>>>> only needs
> >>>>>>>>> to agree on the public API, and we can discuss the
> >>> implementation
> >>>>> at
> >>>>>>>>> leisure in
> >>>>>>>>> a PR...
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> -John
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> >>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
> >>>>>>>>>>> Hello Adam / Jan / John,
> >>>>>>>>>>>
> >>>>>>>>>>> Sorry for being late on this thread! I've finally got some
> >>>>> time
> >>>>>>> this
> >>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
> >>> I've
> >>>>>> also
> >>>>>>>>>> realized
> >>>>>>>>>>> there are a bunch of other things I need to enqueue while
> >>>>>> cleaning
> >>>>>>>> them
> >>>>>>>>>> up
> >>>>>>>>>>> --- sth I need to improve on my side). So here are my
> >>>>> thoughts:
> >>>>>>>>>>>
> >>>>>>>>>>> Regarding the APIs: I like the current written API in the
> >>> KIP.
> >>>>>> More
> >>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
> >>>>>>> functionalities
> >>>>>>>> as
> >>>>>>>>>>> well as 2) other join types than inner as separate KIPs
> >>> since
> >>>>> 1)
> >>>>>>> may
> >>>>>>>>>> worth
> >>>>>>>>>>> a general API refactoring that can benefit not only
> >>> foreignkey
> >>>>>>> joins
> >>>>>>>>> but
> >>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> >>>>>>>>>> ),
> >>>>>>>>>>> and I'm not sure if other join types would actually be
> >>> needed
> >>>>>>> (maybe
> >>>>>>>>> left
> >>>>>>>>>>> join still makes sense), so it's better to
> >>>>>>>>> wait-for-people-to-ask-and-add
> >>>>>>>>>>> than add-sth-that-no-one-uses.
> >>>>>>>>>>>
> >>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
> >>> a
> >>>>>>>>>>> KScatteredTable for users to inject their own optimization:
> >>>>> I'd
> >>>>>>>> prefer
> >>>>>>>>> to
> >>>>>>>>>>> do the current option as-is, and my main rationale is for
> >>>>>>>> optimization
> >>>>>>>>>>> rooms inside the Streams internals and the API
> >>> succinctness.
> >>>>> For
> >>>>>>>>> advanced
> >>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
> >>> own
> >>>>>>>>>> optimization,
> >>>>>>>>>>> while it is too much of the work to use Processor API
> >>>>> directly, I
> >>>>>>>> think
> >>>>>>>>>> we
> >>>>>>>>>>> can still extend the current API to support it in the
> >>> future
> >>>>> if
> >>>>>> it
> >>>>>>>>>> becomes
> >>>>>>>>>>> necessary.
> >>>>>>>>>>
> >>>>>>>>>> no internal optimization potential. it's a myth
> >>>>>>>>>>
> >>>>>>>>>> ¯\_(ツ)_/¯
> >>>>>>>>>>
> >>>>>>>>>> :-)
> >>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
> >>> as
> >>>>> I
> >>>>>>>>> mentioned
> >>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
> >>> key-value
> >>>>>>> store)
> >>>>>>>>> we
> >>>>>>>>>>> can actually make this step simpler than the current
> >>>>> proposal. In
> >>>>>>>> fact,
> >>>>>>>>>> we
> >>>>>>>>>>> can just keep a single final-result store with timestamps
> >>> and
> >>>>>>> reject
> >>>>>>>>>> values
> >>>>>>>>>>> that have a smaller timestamp, is that right?
> >>>>>>>>>>
> >>>>>>>>>> Which is the correct output should at least be decided on the
> >>>>>> offset
> >>>>>>> of
> >>>>>>>>>> the original message.
> >>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
> >>>>> Adam
> >>>>>> to
> >>>>>>>>> make
> >>>>>>>>>>> such HUGE progress on this KIP!
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Guozhang
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> >>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> If they don't find the time:
> >>>>>>>>>>>> They usually take the opposite path from me :D
> >>>>>>>>>>>> so the answer would be clear.
> >>>>>>>>>>>>
> >>>>>>>>>>>> hence my suggestion to vote.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
> >>>>>>>>>>>>> Hi Guozhang and Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
> >>> KIP
> >>>>>> to a
> >>>>>>>>> point
> >>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
> >>> a
> >>>>>>>>> tie-breaker,
> >>>>>>>>>>>> if
> >>>>>>>>>>>>> you will). If you have anyone else you may think should
> >>>>> look at
> >>>>>>>> this,
> >>>>>>>>>>>>> please tag them accordingly.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The scenario is as such:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Current Option:
> >>>>>>>>>>>>> API:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> >>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
> >>>>>>> partition
> >>>>>>>>> with
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> foreignKey (repartition 1)
> >>>>>>>>>>>>> 2) Join the data
> >>>>>>>>>>>>> 3) Shuffle the data back to the original node
> >>> (repartition
> >>>>> 2)
> >>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
> >>>>>>> foreign-key
> >>>>>>>>>>>> changes.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Alternate Option:
> >>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
> >>>>>>>>>>>>> - It would be keyed on a wrapped key function:
> >>>>> <CombinedKey<KO,
> >>>>>>> K>,
> >>>>>>>>> VR>
> >>>>>>>>>>>> (KO
> >>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
> >>> Result)
> >>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
> >>>>>>> otherwise a
> >>>>>>>>>> user
> >>>>>>>>>>>>> would be able to perform additional functions directly
> >>> from
> >>>>> the
> >>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
> >>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
> >>>>> tradeoffs.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
> >>> possible,
> >>>>>> but
> >>>>>>>> will
> >>>>>>>>>>>>> require for implementation details to be made in the API
> >>> and
> >>>>>> some
> >>>>>>>>>>>> exposure
> >>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I appreciate any insight into this.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> >>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi John
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
> >>>>> summary
> >>>>>> is
> >>>>>>>>>>>> accurate
> >>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
> >>> that
> >>>>>>> there
> >>>>>>>>> is a
> >>>>>>>>>>>> risk
> >>>>>>>>>>>>>> of inconsistent final states without performing the
> >>>>>> resolution.
> >>>>>>>> This
> >>>>>>>>>> is
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>> major concern for me as most of the data I have dealt
> >>> with
> >>>>> is
> >>>>>>>>> produced
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>> relational databases. We have seen a number of cases
> >>> where
> >>>>> a
> >>>>>>> user
> >>>>>>>> in
> >>>>>>>>>> the
> >>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
> >>>>> they
> >>>>>>> made
> >>>>>>>> a
> >>>>>>>>>>>>>> mistake, and then updated the field again with a new
> >>> key.
> >>>>> The
> >>>>>>>> events
> >>>>>>>>>> are
> >>>>>>>>>>>>>> propagated out as they are produced, and as such we have
> >>>>> had
> >>>>>>>>>> real-world
> >>>>>>>>>>>>>> cases where these inconsistencies were propagated
> >>>>> downstream
> >>>>>> as
> >>>>>>>> the
> >>>>>>>>>>>> final
> >>>>>>>>>>>>>> values due to the race conditions in the fanout of the
> >>>>> data.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> This solution that I propose values correctness of the
> >>>>> final
> >>>>>>>> result
> >>>>>>>>>> over
> >>>>>>>>>>>>>> other factors.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> We could always move this function over to using a
> >>>>>>> KScatteredTable
> >>>>>>>>>>>>>> implementation in the future, and simply deprecate it
> >>> this
> >>>>>> join
> >>>>>>>> API
> >>>>>>>>> in
> >>>>>>>>>>>>>> time. I think I would like to hear more from some of the
> >>>>> other
> >>>>>>>> major
> >>>>>>>>>>>>>> committers on which course of action they would think is
> >>>>> best
> >>>>>>>> before
> >>>>>>>>>> any
> >>>>>>>>>>>>>> more coding is done.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks again
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> >>>>>> john@confluent.io>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Jan and Adam,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
> >>> are
> >>>>>>>>> encouraging.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
> >>>>> agree
> >>>>>>> that
> >>>>>>>>>>>> avoiding
> >>>>>>>>>>>>>>> unnecessary join outputs is especially important when
> >>> the
> >>>>>>> fan-out
> >>>>>>>>> is
> >>>>>>>>>> so
> >>>>>>>>>>>>>>> high. I suppose this could also be built into the
> >>>>>>> implementation
> >>>>>>>>>> we're
> >>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
> >>>>> KIP
> >>>>>>>> (since
> >>>>>>>>>>>> it's an
> >>>>>>>>>>>>>>> API-transparent optimization).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
> >>>>> didn't
> >>>>>>>> bring
> >>>>>>>>>> it
> >>>>>>>>>>>> up
> >>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
> >>> the
> >>>>>> KIP
> >>>>>>>>> as-is,
> >>>>>>>>>>>>>>> despite the disagreement.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
> >>>>>>>> reasonable.
> >>>>>>>>>>>>>>> It sounds like Jan values more the potential for
> >>>>> developers
> >>>>>> to
> >>>>>>>>>> optimize
> >>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
> >>> whereas
> >>>>>> Adam
> >>>>>>>>>> places
> >>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>> value on having a single operator that people can use
> >>>>> without
> >>>>>>>> extra
> >>>>>>>>>>>> steps
> >>>>>>>>>>>>>>> at the end.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Personally, although I do find it exceptionally
> >>> annoying
> >>>>>> when a
> >>>>>>>>>>>> framework
> >>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
> >>> it
> >>>>>> seems
> >>>>>>>>> better
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>> for a single operation.
> >>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
> >>>>> significant
> >>>>>>>>>> latitude
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>> the implementation (for example, joining only at the
> >>> end,
> >>>>> not
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> middle
> >>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
> >>> resolution;
> >>>>> how
> >>>>>> we
> >>>>>>>>>>>> represent
> >>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
> >>>>> vectors),
> >>>>>>>>> etc.).
> >>>>>>>>>>>> If we
> >>>>>>>>>>>>>>> publish something like a KScatteredTable with the
> >>>>>>>> right-partitioned
> >>>>>>>>>>>> joined
> >>>>>>>>>>>>>>> data, then the API pretty much locks in the
> >>>>> implementation as
> >>>>>>>> well.
> >>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
> >>> mean
> >>>>>>> "seems";
> >>>>>>>>> if
> >>>>>>>>>>>>>>> anyone
> >>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
> >>>>>>> simpler,
> >>>>>>>> I
> >>>>>>>>>>>> think
> >>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
> >>>>> algebra
> >>>>>>>>>>>> perspective,
> >>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
> >>>>> KTable
> >>>>>>> in
> >>>>>>>>> all
> >>>>>>>>>>>>>>> cases.
> >>>>>>>>>>>>>>> * That said, there might still be room in the API for a
> >>>>>>> different
> >>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
> >>> then
> >>>>> do
> >>>>>>>> things
> >>>>>>>>>> like
> >>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
> >>> haven't
> >>>>>>> thought
> >>>>>>>>>>>> through
> >>>>>>>>>>>>>>> all the consequences yet.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> This is all just my opinion after thinking over the
> >>>>>> discussion
> >>>>>>> so
> >>>>>>>>>>>> far...
> >>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> >>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
> >>> the
> >>>>>>>>>> prefixScan.
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>> have attached the file, but I will also include the
> >>> text
> >>>>> in
> >>>>>>> the
> >>>>>>>>> body
> >>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
> >>>>>> attached
> >>>>>>>>>> files).
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
> >>> large
> >>>>>>> values
> >>>>>>>> of
> >>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
> >>>>> would
> >>>>>> be
> >>>>>>>> more
> >>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
> >>>>> For
> >>>>>>>>> instance,
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
> >>> and a
> >>>>>>> further
> >>>>>>>>>>>>>>> ~35-85mS
> >>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
> >>>>> them.
> >>>>>>>>> Iterating
> >>>>>>>>>>>>>>>> through the records just to generate a simple count
> >>>>> takes ~
> >>>>>> 40
> >>>>>>>>> times
> >>>>>>>>>>>>>>> longer
> >>>>>>>>>>>>>>>> than the flush + scan combined.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> ============================================================================================
> >>>>>>>>>>>>>>>> Setup:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> ============================================================================================
> >>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
> >>>>>>> (Xmx512m,
> >>>>>>>>>>>> Xms512m)
> >>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
> >>> directly-accessible
> >>>>>>> Kafka
> >>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
> >>> the
> >>>>>>>>>>>>>>>> ProcessorContext).
> >>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
> >>> values
> >>>>>>>> provided
> >>>>>>>>> in
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> 2.1/trunk release.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> keysize = 128 bytes
> >>>>>>>>>>>>>>>> valsize = 512 bytes
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Step 1:
> >>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
> >>>>>> left-padded
> >>>>>>>>>>>>>>>> auto-incrementing integer)
> >>>>>>>>>>>>>>>> Step 2:
> >>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
> >>>>>>>>>>>> auto-incrementing
> >>>>>>>>>>>>>>>> integer)
> >>>>>>>>>>>>>>>> Step 3:
> >>>>>>>>>>>>>>>> Perform flush
> >>>>>>>>>>>>>>>> Step 4:
> >>>>>>>>>>>>>>>> Perform prefixScan
> >>>>>>>>>>>>>>>> Step 5:
> >>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
> >>> count of
> >>>>>>>> expected
> >>>>>>>>>>>>>>> events.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> ============================================================================================
> >>>>>>>>>>>>>>>> Results:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> ============================================================================================
> >>>>>>>>>>>>>>>> X = 1k (11k events total)
> >>>>>>>>>>>>>>>> Flush Time = 39 mS
> >>>>>>>>>>>>>>>> Scan Time = 7 mS
> >>>>>>>>>>>>>>>> 6.9 MB disk
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>> X = 10k (110k events total)
> >>>>>>>>>>>>>>>> Flush Time = 45 mS
> >>>>>>>>>>>>>>>> Scan Time = 8 mS
> >>>>>>>>>>>>>>>> 127 MB
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>> X = 100k (1.1M events total)
> >>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>> Flush Time = 60 mS
> >>>>>>>>>>>>>>>> Scan Time = 12 mS
> >>>>>>>>>>>>>>>> 678 MB
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>> Flush Time = 45 mS
> >>>>>>>>>>>>>>>> Scan Time = 7 mS
> >>>>>>>>>>>>>>>> 576 MB
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>> X = 1MB (11M events total)
> >>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>> Flush Time = 52 mS
> >>>>>>>>>>>>>>>> Scan Time = 19 mS
> >>>>>>>>>>>>>>>> 7.2 GB
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>> Flush Time = 84 mS
> >>>>>>>>>>>>>>>> Scan Time = 34 mS
> >>>>>>>>>>>>>>>> 9.1 GB
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
> >>>>>>>>>>>>>>>> Test1:
> >>>>>>>>>>>>>>>> Flush Time = 82 mS
> >>>>>>>>>>>>>>>> Scan Time = 63 mS
> >>>>>>>>>>>>>>>> 17GB - 276 sst files
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Test2:
> >>>>>>>>>>>>>>>> Flush Time = 116 mS
> >>>>>>>>>>>>>>>> Scan Time = 35 mS
> >>>>>>>>>>>>>>>> 23GB - 361 sst files
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Test3:
> >>>>>>>>>>>>>>>> Flush Time = 103 mS
> >>>>>>>>>>>>>>>> Scan Time = 82 mS
> >>>>>>>>>>>>>>>> 19 GB - 300 sst files
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
> >>>>> events. I
> >>>>>>>> tried
> >>>>>>>>>> to
> >>>>>>>>>>>> go
> >>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
> >>> the
> >>>>>> 100GB+
> >>>>>>>>> range
> >>>>>>>>>>>>>>> and my
> >>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
> >>> be
> >>>>> done
> >>>>>>>> but I
> >>>>>>>>>>>>>>> suspect
> >>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
> >>>>>> results
> >>>>>>>>>> above.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> At this point in time, I think the only major
> >>> discussion
> >>>>>> point
> >>>>>>>> is
> >>>>>>>>>>>> really
> >>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
> >>> repartitioning
> >>>>>> back +
> >>>>>>>>>>>> resolving
> >>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
> >>> the
> >>>>>> client
> >>>>>>>> to
> >>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks folks,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> >>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> >>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
> >>> the
> >>>>> 2.1
> >>>>>>>>> release
> >>>>>>>>>>>>>>>>> caused an
> >>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
> >>>>> radar
> >>>>>>>> (which
> >>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
> >>>>> some
> >>>>>>>> extend
> >>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
> >>>>> keep) me
> >>>>>>>>>>>>>>> preoccupied.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
> >>> guess
> >>>>> Jan
> >>>>>>> was
> >>>>>>>>> on
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>> track!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
> >>> and
> >>>>> the
> >>>>>>>> prior
> >>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> API Thoughts:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
> >>>>>> many-to-one
> >>>>>>>>> join.
> >>>>>>>>>>>>>>> Could
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
> >>>>> flip
> >>>>>>> the
> >>>>>>>>>> design
> >>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
> >>> join
> >>>>>>> type,
> >>>>>>>>> and
> >>>>>>>>>> it
> >>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
> >>>>>>> one-to-one
> >>>>>>>>>> join.
> >>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>> would work, of course, but it would be super
> >>>>> inefficient
> >>>>>>>>> compared
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> simple rekey-and-join.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
> >>>>>> specified
> >>>>>>>>>> whether
> >>>>>>>>>>>>>>>>> it's an
> >>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
> >>>>> join, as
> >>>>>>>>>>>>>>> (neglecting
> >>>>>>>>>>>>>>>>> IQ),
> >>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
> >>>>> it in
> >>>>>>> the
> >>>>>>>>>>>>>>>>> ValueJoiner.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
> >>> quite
> >>>>>>> right.
> >>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
> >>>>>>> paradigms
> >>>>>>>> in
> >>>>>>>>>>>>>>> play in
> >>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
> >>>>> three
> >>>>>>>>>> Serialized
> >>>>>>>>>>>>>>>>> args, I
> >>>>>>>>>>>>>>>>>> think it would be better to have one that allows
> >>>>>>> (optionally)
> >>>>>>>>>>>> setting
> >>>>>>>>>>>>>>>>> the 4
> >>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
> >>>>>>>>> Materialized.
> >>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
> >>>>> already
> >>>>>>> be
> >>>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
> >>>>> config
> >>>>>>>> might
> >>>>>>>>> be
> >>>>>>>>>>>>>>>>>> applicable.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
> >>> joins
> >>>>>>> don't
> >>>>>>>>>> allow
> >>>>>>>>>>>>>>>>> setting
> >>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
> >>>>> since
> >>>>>>> the
> >>>>>>>>>> rekey
> >>>>>>>>>>>>>>>>>> operation needs to produce results that are
> >>>>> co-partitioned
> >>>>>>>> with
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> "other"
> >>>>>>>>>>>>>>>>>> KTable.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
> >>>>> didn't
> >>>>>>>>> actually
> >>>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
> >>>>>>>> "deserializing"
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
> >>>>> think
> >>>>>>> that
> >>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
> >>>>> namespace
> >>>>>> at
> >>>>>>>>> source
> >>>>>>>>>>>>>>> nodes
> >>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
> >>>>> practically
> >>>>>> no
> >>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
> >>> API
> >>>>>> change
> >>>>>>>> wrt
> >>>>>>>>>>>>>>> headers
> >>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
> >>> placed
> >>>>> on
> >>>>>>>> users'
> >>>>>>>>>>>>>>>>> headers.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
> >>> the
> >>>>>>> header
> >>>>>>>> at
> >>>>>>>>>>>>>>> all...
> >>>>>>>>>>>>>>>>>> stay tuned)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
> >>>>> table
> >>>>>>>>> growing
> >>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
> >>>>>>> implementing
> >>>>>>>>> OCC
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
> >>> the
> >>>>>>> rekey
> >>>>>>>> is
> >>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
> >>> changes.
> >>>>> As
> >>>>>>> such,
> >>>>>>>>> it
> >>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
> >>> offset in
> >>>>>> the
> >>>>>>>>> source
> >>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
> >>> the
> >>>>> same
> >>>>>>>>> number
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>> keys
> >>>>>>>>>>>>>>>>>> as the source table at all times.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
> >>>>> might
> >>>>>> be
> >>>>>>>>>> relevant
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
> >>>>> timestamp
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
> >>> the
> >>>>>>>> timestamp
> >>>>>>>>>> as
> >>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>> OCC
> >>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
> >>>>> want to
> >>>>>>> use
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
> >>> use a
> >>>>>>> similar
> >>>>>>>>>>>>>>>>> mechanism to
> >>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
> >>>>>> re-keyed
> >>>>>>>>>> values,
> >>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>> you could avoid a separate table.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
> >>> long
> >>>>>>> time,
> >>>>>>>> so
> >>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
> >>> if we
> >>>>>> can
> >>>>>>>>> avoid
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> HWM
> >>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
> >>> final
> >>>>>> join
> >>>>>>>>>>>>>>> instead...
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
> >>> Letter
> >>>>>> FK,
> >>>>>>>>> (other
> >>>>>>>>>>>>>>>>> data))
> >>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Left table:
> >>>>>>>>>>>>>>>>>> 1: (A, xyz)
> >>>>>>>>>>>>>>>>>> 2: (B, asd)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Right table:
> >>>>>>>>>>>>>>>>>> A: EntityA
> >>>>>>>>>>>>>>>>>> B: EntityB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
> >>>>> key,
> >>>>>> but
> >>>>>>>> not
> >>>>>>>>>>>>>>>>>> propagating the value at all..
> >>>>>>>>>>>>>>>>>> Rekey table:
> >>>>>>>>>>>>>>>>>> A-1: (dummy value)
> >>>>>>>>>>>>>>>>>> B-2: (dummy value)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
> >>>>>>>>>>>>>>>>>> A-1: EntityA
> >>>>>>>>>>>>>>>>>> B-2: EntityB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Which gets rekeyed back:
> >>>>>>>>>>>>>>>>>> 1: A, EntityA
> >>>>>>>>>>>>>>>>>> 2: B, EntityB
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> And finally we do the actual join:
> >>>>>>>>>>>>>>>>>> Result table:
> >>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
> >>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
> >>>>>> opportunity
> >>>>>>>> to
> >>>>>>>>>>>>>>> compare
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
> >>>>> the
> >>>>>>> right
> >>>>>>>>>>>>>>> table. If
> >>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
> >>>>> must be
> >>>>>>>>>> outdated.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
> >>> xyz)
> >>>>> gets
> >>>>>>>>> updated
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> (1:
> >>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
> >>> whether
> >>>>> the
> >>>>>>>> final
> >>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
> >>>>>> algorithm
> >>>>>>>>> above,
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
> >>>>> (B,
> >>>>>>> xyz),
> >>>>>>>>> (B,
> >>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
> >>>>>>> information
> >>>>>>>>> to
> >>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
> >>>>> mentioned
> >>>>>>>> often
> >>>>>>>>> I
> >>>>>>>>>> am
> >>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
> >>>>>>> automatically.
> >>>>>>>> I
> >>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>> this is the real performance eater ;)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
> >>>>>>>> performance
> >>>>>>>>> of
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> range scans when records change in the right table.
> >>>>> You've
> >>>>>>>> said
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>>> you've
> >>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
> >>>>> for a
> >>>>>>>>> while.
> >>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
> >>>>> you've
> >>>>>>>>>> observed?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
> >>>>> topmost
> >>>>>>>> thing
> >>>>>>>>>> here
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
> >>>>>>> performance
> >>>>>>>>> of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
> >>>>> 500k -
> >>>>>>> 1kk
> >>>>>>>>> rows
> >>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
> >>> the
> >>>>> work
> >>>>>>> you
> >>>>>>>>>> gotta
> >>>>>>>>>>>>>>>>> do, also when you pack your data into different
> >>> formats,
> >>>>>>>> usually
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
> >>> data
> >>>>> and
> >>>>>>> we
> >>>>>>>>>> can't
> >>>>>>>>>>>>>>>>> really change that. It is more important for users to
> >>>>>> prevent
> >>>>>>>>>> useless
> >>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
> >>> to
> >>>>> drop
> >>>>>>>>> changes
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> are not going to change my join output.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> usually it's:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
> >>>>>> old.equals(new)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
> >>> for
> >>>>>>>> smaller
> >>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
> >>> then
> >>>>>>> anyways.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
> >>>>> sure if
> >>>>>>>> it's
> >>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
> >>> the
> >>>>>>>> original
> >>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>> as I
> >>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
> >>> keys in
> >>>>>> the
> >>>>>>>>> value:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Left table:
> >>>>>>>>>>>>>>>>>> 1: A,...
> >>>>>>>>>>>>>>>>>> 2: B,...
> >>>>>>>>>>>>>>>>>> 3: A,...
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Gets re-keyed:
> >>>>>>>>>>>>>>>>>> A: [1, 3]
> >>>>>>>>>>>>>>>>>> B: [2]
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
> >>>>> regular
> >>>>>>>>>> single-key
> >>>>>>>>>>>>>>>>> lookup.
> >>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
> >>>>>> values,
> >>>>>>> as
> >>>>>>>>>>>>>>> there's
> >>>>>>>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
> >>> reference
> >>>>> rhs
> >>>>>>>>> records.
> >>>>>>>>>>>>>>>>> Offhand,
> >>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
> >>>>> past
> >>>>>>> the
> >>>>>>>>>>>>>>>>> threshold, we
> >>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
> >>> cases,
> >>>>> it
> >>>>>>> would
> >>>>>>>>> be
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
> >>> be
> >>>>> one
> >>>>>>> per
> >>>>>>>>>> (max
> >>>>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
> >>>>> there's
> >>>>>>> some
> >>>>>>>>>> extra
> >>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
> >>>>>> retraction.
> >>>>>>>> For
> >>>>>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
> >>> just
> >>>>> have
> >>>>>>> (A:
> >>>>>>>>>> [3]).
> >>>>>>>>>>>>>>>>> Some
> >>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
> >>>>> for 1
> >>>>>>> can
> >>>>>>>>> also
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> retracted.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> That's all!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
> >>> thoughtful
> >>>>>> KIP.
> >>>>>>>>> Sorry
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> discussion has been slow.
> >>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> >>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
> >>> up,
> >>>>> it
> >>>>>>> just
> >>>>>>>>>> goes
> >>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>> to discuss.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> would not expect to much attention with another
> >>>>> another
> >>>>>>> email
> >>>>>>>>> in
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> best Jan
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>> Hello Contributors
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
> >>>>> need
> >>>>>> to
> >>>>>>>> bump
> >>>>>>>>>>>>>>> this to
> >>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
> >>>>> through
> >>>>>>>> once
> >>>>>>>>>>>>>>>>> contributor
> >>>>>>>>>>>>>>>>>>>> feedback is given.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Main points that need addressing:
> >>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
> >>>>>>> singular
> >>>>>>>>>> graph
> >>>>>>>>>>>>>>> node
> >>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
> >>> right
> >>>>>>> now. I
> >>>>>>>>> am
> >>>>>>>>>> a
> >>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>>> fuzzy
> >>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
> >>> I
> >>>>>> would
> >>>>>>>>>>>>>>> appreciate
> >>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> help on this aspect.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
> >>>>> thread
> >>>>>>> has
> >>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>> discourse
> >>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
> >>> mark
> >>>>>>>> proposal
> >>>>>>>>>> and
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> groupBy
> >>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
> >>> need
> >>>>> to
> >>>>>>>>> strictly
> >>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
> >>> up
> >>>>> to
> >>>>>> the
> >>>>>>>>>>>>>>> consumer.
> >>>>>>>>>>>>>>>>> Any
> >>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
> >>> this
> >>>>>> from
> >>>>>>>>>> moving
> >>>>>>>>>>>>>>> to a
> >>>>>>>>>>>>>>>>>>> vote?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
> >>>>>>>>>>>>>>> Stores.persistentWindowStore,
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
> >>> segments
> >>>>>> you
> >>>>>>>> want
> >>>>>>>>>> and
> >>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>> large
> >>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
> >>>>> happens
> >>>>>>> is
> >>>>>>>>> that
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
> >>> data
> >>>>> with
> >>>>>>> new
> >>>>>>>>>>>>>>>>> timestamps
> >>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
> >>> of
> >>>>> the
> >>>>>>>> work
> >>>>>>>>>> done
> >>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
> >>> the
> >>>>>>>> hopping
> >>>>>>>>>>>>>>> windowed
> >>>>>>>>>>>>>>>>>>> store.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
> >>>>> to be
> >>>>>>> two
> >>>>>>>>> 24h
> >>>>>>>>>>>>>>>>>>> segments.
> >>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
> >>> will
> >>>>> be
> >>>>>>>>> inserted
> >>>>>>>>>> at
> >>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>> T in
> >>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
> >>> overlap
> >>>>> by
> >>>>>>> 12h.
> >>>>>>>>> As
> >>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>> goes on
> >>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
> >>> the
> >>>>>>> oldest
> >>>>>>>>>>>> segment
> >>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
> >>> The
> >>>>>>>> records
> >>>>>>>>>> are
> >>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
> >>> it
> >>>>> is
> >>>>>>> the
> >>>>>>>>>> record
> >>>>>>>>>>>>>>>>> time,
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
> >>> are
> >>>>>>>> retained
> >>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
> >>> TTL
> >>>>>> at a
> >>>>>>>>>>>>>>>>> segment-level,
> >>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
> >>>>>>>>>> ClassCastException
> >>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>> similar.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
> >>>>>>> fetch/put
> >>>>>>>> or
> >>>>>>>>>>>>>>> iterate
> >>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
> >>>>> stream-time
> >>>>>>>> under
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> hoods
> >>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
> >>> all
> >>>>> the
> >>>>>>>>> windows
> >>>>>>>>>>>>>>>>> earlier?
> >>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
> >>>>> time
> >>>>>>>>>> gurantees.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Best
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
> >>>>> changed
> >>>>>>> the
> >>>>>>>>>> state
> >>>>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
> >>>>>> something
> >>>>>>>> that
> >>>>>>>>>>>>>>> will be
> >>>>>>>>>>>>>>>>>>>>>>>>> extremely
> >>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
> >>> said, I
> >>>>>> will
> >>>>>>>> not
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>>> moving
> >>>>>>>>>>>>>>>>>>>>>> ahead
> >>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
> >>> reshuffle/groupBy
> >>>>>>> solution
> >>>>>>>>> as
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>> propose.
> >>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
> >>>>>> yourself
> >>>>>>>> off
> >>>>>>>>>> of
> >>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>> current PR
> >>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
> >>>>> would
> >>>>>>> be
> >>>>>>>>> more
> >>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>> happy to
> >>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> >>>>>> stands
> >>>>>>>>> right
> >>>>>>>>>>>>>>> now,
> >>>>>>>>>>>>>>>>> I do
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
> >>>>> alternatives
> >>>>>>>> without
> >>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
> >>> which
> >>>>>> they
> >>>>>>>>> would
> >>>>>>>>>>>>>>>>> prefer.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
> >>> gave
> >>>>> up
> >>>>>> on
> >>>>>>>> the
> >>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>>> DSL
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
> >>> pulled
> >>>>>> this
> >>>>>>>> KIP
> >>>>>>>>>>>>>>> through
> >>>>>>>>>>>>>>>>>>>>>> already.
> >>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
> >>> based on
> >>>>>>> PAPI.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
> >>>>> windowed
> >>>>>>>> state
> >>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
> >>> then I
> >>>>>> will
> >>>>>>>> come
> >>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
> >>> any of
> >>>>>> the
> >>>>>>>>>> binding
> >>>>>>>>>>>>>>>>> voters
> >>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
> >>>>> updated
> >>>>>> it
> >>>>>>>>>>>>>>> according
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
> >>> windowed
> >>>>>>> store.
> >>>>>>>>>> This
> >>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
> >>> they
> >>>>>> are
> >>>>>>>>>>>>>>> completed.
> >>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> >>>>>> already
> >>>>>>>>>> updated
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> PR?
> >>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
> >>> Missing
> >>>>>>>>> something?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
> >>> Wang <
> >>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
> >>> is
> >>>>> the
> >>>>>>>> wrong
> >>>>>>>>>>>> link,
> >>>>>>>>>>>>>>>>> as it
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
> >>>>> part of
> >>>>>>>>> KIP-258
> >>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
> >>>>>> KTable"
> >>>>>>>> such
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
> >>> materialized
> >>>>>> store,
> >>>>>>>>> i.e.
> >>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
> >>> older
> >>>>>> than
> >>>>>>>> the
> >>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>> key's
> >>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
> >>> ordering.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
> >>>>> Wang <
> >>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
> >>>>> final
> >>>>>>> step
> >>>>>>>>>> (i.e.
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> high
> >>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
> >>>>> with
> >>>>>> a
> >>>>>>>>> window
> >>>>>>>>>>>>>>>>> store),
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
> >>>>> help:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
> >>>>> key-value
> >>>>>>>> store
> >>>>>>>>>>>>>>> (i.e.
> >>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
> >>>>> usage,
> >>>>>> as
> >>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>> https://issues.apache.org/jira/browse/KAFKA-5533
> >>>>>> ,
> >>>>>>> is
> >>>>>>>>>> that
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
> >>> its
> >>>>>>>>> timestamp
> >>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> smaller
> >>>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
> >>> I
> >>>>>> think
> >>>>>>> it
> >>>>>>>>> is
> >>>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
> >>> based
> >>>>>>>>> filtering,
> >>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
> >>> the
> >>>>>>> joining
> >>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> correctly
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
> >>>>> final
> >>>>>>>> stage.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
> >>> and
> >>>>>> hence
> >>>>>>>>>>>>>>>>> non-windowed
> >>>>>>>>>>>>>>>>>>>>>> KTables
> >>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
> >>>>> really
> >>>>>>> have
> >>>>>>>> a
> >>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>> support
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
> >>>>>>>>>>>>>>>>>>>>>>
> >>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
> >>>>>>> KTable-KTable
> >>>>>>>>>>>>>>> non-key
> >>>>>>>>>>>>>>>>>>> joins
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
> >>> Filipiak
> >>>>> <
> >>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
> >>> would
> >>>>>> grow
> >>>>>>>>>>>> endlessly
> >>>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
> >>> pair
> >>>>> of
> >>>>>>>> (<this
> >>>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
> >>> is
> >>>>> used
> >>>>>>> to
> >>>>>>>>>>>>>>>>> differentiate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
> >>>>> proposal
> >>>>>>>> would
> >>>>>>>>>> be
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> replace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
> >>>>> This
> >>>>>>> would
> >>>>>>>>>> allow
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
> >>> time.
> >>>>> This
> >>>>>>>>> should
> >>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> >>>>>> should
> >>>>>>> be
> >>>>>>>>>>>>>>>>> customizable
> >>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
> >>>>> perhaps
> >>>>>>> just
> >>>>>>>>> 10
> >>>>>>>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
> >>> do
> >>>>> the
> >>>>>>>> trick
> >>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>> Even
> >>>>>>>>>>>>>>>>>>>>>> if I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
> >>>>>>> repartitioning
> >>>>>>>>>>>>>>> optional
> >>>>>>>>>>>>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
> >>>>> am a
> >>>>>>>> little
> >>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>> sceptical
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
> >>> one
> >>>>>>> could
> >>>>>>>>> run
> >>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
> >>>>> border. I
> >>>>>>> will
> >>>>>>>>>> check
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
> >>>>> problematic, we
> >>>>>>>> could
> >>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
> >>> performance
> >>>>>>> impact I
> >>>>>>>>>>>> guess.
> >>>>>>>>>>>>>>>>> Will
> >>>>>>>>>>>>>>>>>>>>>> let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
> >>> correct
> >>>>> as
> >>>>>>> is. I
> >>>>>>>>>>>>>>> wouldn't
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> >>>>>>> timestamp(A)  <
> >>>>>>>>>>>>>>>>>>> timestamp(B).
> >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
> >>> -
> >>>>>> thanks
> >>>>>>>> for
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> diagram, it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
> >>> do
> >>>>> not
> >>>>>>> have
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
> >>>>>>> available
> >>>>>>>>>> then
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>> would be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
> >>> Map.
> >>>>>> That
> >>>>>>>>> being
> >>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
> >>> for
> >>>>>>> clarity
> >>>>>>>>> for
> >>>>>>>>>>>>>>>>> everyone
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
> >>> really
> >>>>> hard
> >>>>>>>> work.
> >>>>>>>>>> But
> >>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> >>>>>>> original
> >>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>> key,
> >>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
> >>> PAPI
> >>>>>> and
> >>>>>>>>>>>> basically
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
> >>> missed
> >>>>>> that
> >>>>>>> in
> >>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>> DSL
> >>>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
> >>>>> up on
> >>>>>>> my
> >>>>>>>>> end.
> >>>>>>>>>>>>>>> Will
> >>>>>>>>>>>>>>>>>>>>>> finish
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
> >>>>> week.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
> >>> the
> >>>>> Map
> >>>>>>> stay
> >>>>>>>>>>>> inside
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> State
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
> >>> changes
> >>>>>> have
> >>>>>>>>>>>>>>> propagated?
> >>>>>>>>>>>>>>>>>>> Isn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
> >>>>> state
> >>>>>>>> store?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
> >>>>> substractor
> >>>>>> is
> >>>>>>>>> gonna
> >>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>> `null`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
> >>>>> there
> >>>>>> is
> >>>>>>>>> going
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
> >>> this
> >>>>>> store
> >>>>>>>>>> directly
> >>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
> >>> is a
> >>>>>>>> regular
> >>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>> satisfying
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
> >>>>> join.
> >>>>>>> The
> >>>>>>>>>>>>>>> Windowed
> >>>>>>>>>>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
> >>> statefull
> >>>>>>>> operation
> >>>>>>>>>> we
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
> >>>>> have
> >>>>>> the
> >>>>>>>>>> window
> >>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
> >>> custom
> >>>>>> group
> >>>>>>>> by
> >>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
> >>> key i
> >>>>>> think
> >>>>>>>> it
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> help
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
> >>> the
> >>>>>>>> original
> >>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
> >>>>> foundation
> >>>>>>> to
> >>>>>>>>>> build
> >>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
> >>> user.
> >>>>>> very
> >>>>>>>>>>>>>>>>> unfortunate. I
> >>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
> >>> do
> >>>>> not
> >>>>>>>> think
> >>>>>>>>>> its
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
> >>> Prajakta
> >>>>>>> Dumbre <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            please remove me from this
> >>> group
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            On Tue, Sep 11, 2018 at 1:29 PM
> >>>>> Jan
> >>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
> >>>>> <mailto:
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Hi Adam,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > give me some time, will make
> >>>>> such a
> >>>>>>>>> chart.
> >>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>> time i
> >>>>>>>>>>>>>>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            get along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > well with giphy and ruined
> >>> all
> >>>>> your
> >>>>>>>>> charts.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Hopefully i can get it done
> >>>>> today
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > On 08.09.2018 16:00, Adam
> >>>>> Bellemare
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > Hi Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > I have included a diagram
> >>> of
> >>>>>> what I
> >>>>>>>>>>>> attempted
> >>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > I attempted this back at
> >>> the
> >>>>>> start
> >>>>>>> of
> >>>>>>>>> my
> >>>>>>>>>> own
> >>>>>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > solution, and since I could
> >>>>> not
> >>>>>> get
> >>>>>>>> it
> >>>>>>>>> to
> >>>>>>>>>>>>>>> work I
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            discarded the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > code. At this point in
> >>> time,
> >>>>> if
> >>>>>> you
> >>>>>>>>> wish
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>> pursuing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            for your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > groupBy solution, I ask
> >>> that
> >>>>> you
> >>>>>>>> please
> >>>>>>>>>>>>>>> create a
> >>>>>>>>>>>>>>>>>>>>>> diagram on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > carefully explaining your
> >>>>>> solution.
> >>>>>>>>>> Please
> >>>>>>>>>>>>>>> feel
> >>>>>>>>>>>>>>>>> free
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the image I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > just posted as a starting
> >>>>> point.
> >>>>>> I
> >>>>>>> am
> >>>>>>>>>> having
> >>>>>>>>>>>>>>>>> trouble
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            understanding your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > explanations but I think
> >>> that
> >>>>> a
> >>>>>>>>> carefully
> >>>>>>>>>>>>>>>>> constructed
> >>>>>>>>>>>>>>>>>>>>>> diagram
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            will clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > any misunderstandings.
> >>>>>> Alternately,
> >>>>>>>>>> please
> >>>>>>>>>>>>>>> post a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            comprehensive PR with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > your solution. I can only
> >>>>> guess
> >>>>>> at
> >>>>>>>> what
> >>>>>>>>>> you
> >>>>>>>>>>>>>>>>> mean, and
> >>>>>>>>>>>>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            value my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > own
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > time as much as you value
> >>>>> yours,
> >>>>>> I
> >>>>>>>>>> believe
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            responsibility to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > provide an implementation
> >>>>> instead
> >>>>>>> of
> >>>>>>>> me
> >>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>> guess.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > On Sat, Sep 8, 2018 at 8:00
> >>>>> AM,
> >>>>>> Jan
> >>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
> >>>>> <mailto:
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Hi James,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> nice to see you beeing
> >>>>>> interested.
> >>>>>>>>> kafka
> >>>>>>>>>>>>>>>>> streams at
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            point supports
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> all sorts of joins as
> >>> long as
> >>>>>> both
> >>>>>>>>>> streams
> >>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Adam is currently
> >>>>> implementing a
> >>>>>>>> join
> >>>>>>>>>>>> where a
> >>>>>>>>>>>>>>>>> KTable
> >>>>>>>>>>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            KTable can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> a one to many relation
> >>> ship
> >>>>>> (1:n).
> >>>>>>>> We
> >>>>>>>>>>>> exploit
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> rocksdb
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> datastore that keeps data
> >>>>> sorted
> >>>>>> (At
> >>>>>>>>> least
> >>>>>>>>>>>>>>>>> exposes an
> >>>>>>>>>>>>>>>>>>>>>> API to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> stored data in a sorted
> >>>>>> fashion).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> I think the technical
> >>> caveats
> >>>>>> are
> >>>>>>>> well
> >>>>>>>>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > basically
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> down to philosophy and API
> >>>>>> Design
> >>>>>>> (
> >>>>>>>>> when
> >>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>> sees
> >>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>> newest
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            message).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> I have a lengthy track
> >>>>> record of
> >>>>>>>>> loosing
> >>>>>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>> kinda
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            arguments within
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> streams community and I
> >>> have
> >>>>> no
> >>>>>>> clue
> >>>>>>>>>> why.
> >>>>>>>>>>>> So
> >>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> literally
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            can't wait for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> to churn through this
> >>> thread
> >>>>> and
> >>>>>>>> give
> >>>>>>>>>> you
> >>>>>>>>>>>>>>>>> opinion on
> >>>>>>>>>>>>>>>>>>>>>> how we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > design
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> the return type of the
> >>>>>>> oneToManyJoin
> >>>>>>>>> and
> >>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>>> power we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            want to give
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> the user vs "simplicity"
> >>>>> (where
> >>>>>>>>>> simplicity
> >>>>>>>>>>>>>>> isn't
> >>>>>>>>>>>>>>>>>>>>>> really that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            as users
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> need to understand it I
> >>>>> argue)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> waiting for you to join
> >>> in on
> >>>>>> the
> >>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Best Jan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> On 07.09.2018 15:49, James
> >>>>> Kwan
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> I am new to this group
> >>> and I
> >>>>>>> found
> >>>>>>>>> this
> >>>>>>>>>>>>>>> subject
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            interesting.  Sounds
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> you guys want to
> >>> implement a
> >>>>>> join
> >>>>>>>>>> table of
> >>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>>>>>> streams? Is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > somewhere
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> I can see the original
> >>>>>>> requirement
> >>>>>>>> or
> >>>>>>>>>>>>>>> proposal?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> On Sep 7, 2018, at 8:13
> >>> AM,
> >>>>> Jan
> >>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
> >>>>> <mailto:
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> On 05.09.2018 22:17,
> >>> Adam
> >>>>>>>> Bellemare
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> I'm currently testing
> >>>>> using a
> >>>>>>>>>> Windowed
> >>>>>>>>>>>>>>> Store
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> store the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            highwater
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> mark.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> By all indications this
> >>>>>> should
> >>>>>>>> work
> >>>>>>>>>>>> fine,
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> caveat
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            being that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> only resolve
> >>> out-of-order
> >>>>>>> arrival
> >>>>>>>>>> for up
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> size of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the window
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > (ie:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> 24h, 72h, etc). This
> >>> would
> >>>>>>> remove
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> possibility
> >>>>>>>>>>>>>>>>>>>>>> of it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > unbounded
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> size.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> With regards to Jan's
> >>>>>>>> suggestion, I
> >>>>>>>>>>>>>>> believe
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> remain in disagreement.
> >>>>>> While I
> >>>>>>>> do
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>>> disagree
> >>>>>>>>>>>>>>>>>>>>>> with your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            statement
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> there likely to be
> >>>>> additional
> >>>>>>>> joins
> >>>>>>>>>> done
> >>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>>>>> real-world
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            workflow, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> see how you can
> >>>>> conclusively
> >>>>>>> deal
> >>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> foreign-key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> changes and subsequent
> >>>>>> joins. I
> >>>>>>>>> have
> >>>>>>>>>>>>>>>>> attempted
> >>>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            think you have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> proposed (without a
> >>>>>> high-water,
> >>>>>>>>> using
> >>>>>>>>>>>>>>>>> groupBy and
> >>>>>>>>>>>>>>>>>>>>>> reduce)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            and found
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> that if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> the foreign key changes
> >>>>> too
> >>>>>>>>> quickly,
> >>>>>>>>>> or
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> load
> >>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            stream thread
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> too
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> high, the joined
> >>> messages
> >>>>>> will
> >>>>>>>>> arrive
> >>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>> and be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            incorrectly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> propagated, such that
> >>> an
> >>>>>>>>> intermediate
> >>>>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > final
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> event.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> Can you shed some light
> >>> on
> >>>>>> your
> >>>>>>>>>> groupBy
> >>>>>>>>>>>>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            There must be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> some sort of flaw in it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> I have a suspicion
> >>> where it
> >>>>>> is,
> >>>>>>> I
> >>>>>>>>>> would
> >>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            confirm. The idea
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> is bullet proof and it
> >>>>> must be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> an implementation mess
> >>> up.
> >>>>> I
> >>>>>>> would
> >>>>>>>>>> like
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> clarify
> >>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we draw a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> conclusion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>    Repartitioning the
> >>>>>> scattered
> >>>>>>>>> events
> >>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> partitions is the only
> >>> way I
> >>>>>> know
> >>>>>>>> how
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> conclusively
> >>>>>>>>>>>>>>>>>>>>>> deal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> out-of-order events in
> >>> a
> >>>>>> given
> >>>>>>>> time
> >>>>>>>>>>>> frame,
> >>>>>>>>>>>>>>>>> and to
> >>>>>>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > data
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> eventually consistent
> >>> with
> >>>>>> the
> >>>>>>>>> input
> >>>>>>>>>>>>>>> events.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> If you have some code
> >>> to
> >>>>>> share
> >>>>>>>> that
> >>>>>>>>>>>>>>>>> illustrates
> >>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            approach, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> very grateful as it
> >>> would
> >>>>>>> remove
> >>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>> misunderstandings
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that I may
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> ah okay you were looking
> >>>>> for
> >>>>>> my
> >>>>>>>>> code.
> >>>>>>>>>> I
> >>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            something easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> readable here as its
> >>>>> bloated
> >>>>>>> with
> >>>>>>>>>>>>>>> OO-patterns.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> its anyhow trivial:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> @Override
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      public T apply(K
> >>>>> aggKey,
> >>>>>> V
> >>>>>>>>>> value, T
> >>>>>>>>>>>>>>>>>>> aggregate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          Map<U, V>
> >>>>>>>>> currentStateAsMap =
> >>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            imaginary
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          U toModifyKey =
> >>>>>>>>>>>>>>> mapper.apply(value);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << this is
> >>> the
> >>>>>>> place
> >>>>>>>>>> where
> >>>>>>>>>>>>>>> people
> >>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            gonna have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > issues
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> and why you probably
> >>>>> couldn't
> >>>>>> do
> >>>>>>>> it.
> >>>>>>>>>> we
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>> to find
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            a solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> I didn't realize that
> >>> yet.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << we
> >>>>> propagate
> >>>>>> the
> >>>>>>>>>> field in
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> joiner, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that we can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > pick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> it up in an aggregate.
> >>>>>> Probably
> >>>>>>>> you
> >>>>>>>>>> have
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>> thought
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            this in your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> approach right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << I am
> >>> very
> >>>>> open
> >>>>>>> to
> >>>>>>>>>> find a
> >>>>>>>>>>>>>>>>> generic
> >>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            here. In my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> honest opinion this is
> >>>>> broken
> >>>>>> in
> >>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            looses
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the keys
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> and only maintains the
> >>>>>> aggregate
> >>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << I
> >>>>> abstracted
> >>>>>> it
> >>>>>>>> away
> >>>>>>>>>> back
> >>>>>>>>>>>>>>>>> then way
> >>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> of oneToMany join. That
> >>> is
> >>>>>> why I
> >>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>> realize
> >>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            significance here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              <<
> >>> Opinions?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          for (V m :
> >>>>> current)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>> currentStateAsMap.put(mapper.apply(m),
> >>>>>>>>>>>> m);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          if (isAdder)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>> currentStateAsMap.put(toModifyKey,
> >>>>>>>>>>>> value);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          else
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>                  return
> >>>>> null;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          retrun
> >>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> On Wed, Sep 5, 2018 at
> >>>>> 3:35
> >>>>>> PM,
> >>>>>>>> Jan
> >>>>>>>>>>>>>>> Filipiak
> >>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Jan.Filipiak@trivago.com
> >>>>> <mailto:
> >>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> Thanks Adam for
> >>> bringing
> >>>>>>> Matthias
> >>>>>>>>> to
> >>>>>>>>>>>>>>> speed!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> about the
> >>> differences. I
> >>>>>> think
> >>>>>>>>>>>> re-keying
> >>>>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            optional at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> best.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I would say we return
> >>> a
> >>>>>>>>>> KScatteredTable
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> reshuffle()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>> KTable<originalKey,Joined>
> >>>>>> to
> >>>>>>>> make
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            repartitioning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> optional.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I am also in a big
> >>>>> favour of
> >>>>>>>> doing
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> group
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> by instead high water
> >>>>> mark
> >>>>>>>>> tracking.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> Just because unbounded
> >>>>>> growth
> >>>>>>> is
> >>>>>>>>>> just
> >>>>>>>>>>>>>>> scary
> >>>>>>>>>>>>>>>>> + It
> >>>>>>>>>>>>>>>>>>>>>> saves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the header
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> stuff.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I think the
> >>> abstraction
> >>>>> of
> >>>>>>>> always
> >>>>>>>>>>>>>>>>> repartitioning
> >>>>>>>>>>>>>>>>>>>>>> back is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            just not so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> strong. Like the work
> >>> has
> >>>>>> been
> >>>>>>>>> done
> >>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            back and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> grouping
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> by something else
> >>>>> afterwards
> >>>>>>> is
> >>>>>>>>>> really
> >>>>>>>>>>>>>>>>> common.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> On 05.09.2018 13:49,
> >>> Adam
> >>>>>>>>> Bellemare
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> Hi Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Thank you for your
> >>>>>> feedback,
> >>>>>>> I
> >>>>>>>> do
> >>>>>>>>>>>>>>>>> appreciate
> >>>>>>>>>>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> While name spacing
> >>>>> would be
> >>>>>>>>>> possible,
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > deserialize
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> user headers what
> >>>>> implies
> >>>>>> a
> >>>>>>>>>> runtime
> >>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            suggest to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> namespace for now to
> >>>>> avoid
> >>>>>>> the
> >>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > problem in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the future, we can
> >>>>> still
> >>>>>> add
> >>>>>>>>> name
> >>>>>>>>>>>>>>> spacing
> >>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Agreed. I will go
> >>> with
> >>>>>>> using a
> >>>>>>>>>>>> reserved
> >>>>>>>>>>>>>>>>> string
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            document it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> My main concern about
> >>>>> the
> >>>>>>>> design
> >>>>>>>>> it
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> type of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            result KTable:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> understood the
> >>> proposal
> >>>>>>>>> correctly,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> In your example, you
> >>>>> have
> >>>>>>>> table1
> >>>>>>>>>> and
> >>>>>>>>>>>>>>> table2
> >>>>>>>>>>>>>>>>>>>>>> swapped.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            Here is how it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> works
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> currently:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 1) table1 has the
> >>>>> records
> >>>>>>> that
> >>>>>>>>>> contain
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> foreign key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            within their
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> value.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 input stream:
> >>>>>>>>>> <a,(fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table2 input stream:
> >>>>> <A,X>,
> >>>>>>>> <B,Y>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 2) A Value mapper is
> >>>>>> required
> >>>>>>>> to
> >>>>>>>>>>>> extract
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> foreign
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 foreign key
> >>>>> mapper:
> >>>>>> (
> >>>>>>>>> value
> >>>>>>>>>> =>
> >>>>>>>>>>>>>>>>> value.fk
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <http://value.fk> )
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> The mapper is
> >>> applied to
> >>>>>> each
> >>>>>>>>>> element
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> table1,
> >>>>>>>>>>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            new combined
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> key is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> made:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 mapped: <A-a,
> >>>>>>>>> (fk=A,bar=1)>,
> >>>>>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <B-c,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 3) The rekeyed events
> >>>>> are
> >>>>>>>>>>>> copartitioned
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> table2:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> a) Stream Thread with
> >>>>>>> Partition
> >>>>>>>>> 0:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> RepartitionedTable1:
> >>>>> <A-a,
> >>>>>>>>>>>>>>> (fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            (fk=A,bar=2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Table2: <A,X>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> b) Stream Thread with
> >>>>>>> Partition
> >>>>>>>>> 1:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> RepartitionedTable1:
> >>>>> <B-c,
> >>>>>>>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 4) From here, they
> >>> can
> >>>>> be
> >>>>>>>> joined
> >>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>> locally
> >>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            applying the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> joiner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> function.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> At this point, Jan's
> >>>>> design
> >>>>>>> and
> >>>>>>>>> my
> >>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>>> deviate. My
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            design goes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> repartition the data
> >>>>>>> post-join
> >>>>>>>>> and
> >>>>>>>>>>>>>>> resolve
> >>>>>>>>>>>>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            arrival of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> records,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> finally returning the
> >>>>> data
> >>>>>>>> keyed
> >>>>>>>>>> just
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> original key.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            I do not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> expose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> CombinedKey or any of
> >>>>> the
> >>>>>>>>> internals
> >>>>>>>>>>>>>>>>> outside of
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            joinOnForeignKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> function. This does
> >>> make
> >>>>>> for
> >>>>>>>>> larger
> >>>>>>>>>>>>>>>>> footprint,
> >>>>>>>>>>>>>>>>>>>>>> but it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            removes all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> agency
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> for resolving
> >>>>> out-of-order
> >>>>>>>>> arrivals
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            CombinedKeys from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> user. I believe that
> >>>>> this
> >>>>>>> makes
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> function
> >>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to use.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Let me know if this
> >>>>> helps
> >>>>>>>> resolve
> >>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>> questions,
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            please feel
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> free to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> add anything else on
> >>>>> your
> >>>>>>> mind.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Thanks again,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> On Tue, Sep 4, 2018
> >>> at
> >>>>> 8:36
> >>>>>>> PM,
> >>>>>>>>>>>>>>> Matthias J.
> >>>>>>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>> matthias@confluent.io
> >>>>>>> <mailto:
> >>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> I am just catching
> >>> up
> >>>>> on
> >>>>>>> this
> >>>>>>>>>>>> thread. I
> >>>>>>>>>>>>>>>>> did
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> read
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            everything so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> far,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> but want to share
> >>>>> couple
> >>>>>> of
> >>>>>>>>>> initial
> >>>>>>>>>>>>>>>>> thoughts:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Headers: I think
> >>> there
> >>>>> is
> >>>>>> a
> >>>>>>>>>>>> fundamental
> >>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            between header
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> usage
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> in this KIP and
> >>> KP-258.
> >>>>>> For
> >>>>>>>> 258,
> >>>>>>>>>> we
> >>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>> headers
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            changelog topic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> are owned by Kafka
> >>>>> Streams
> >>>>>>> and
> >>>>>>>>>> nobody
> >>>>>>>>>>>>>>>>> else is
> >>>>>>>>>>>>>>>>>>>>>> supposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> them. In fact, no
> >>> user
> >>>>>>> header
> >>>>>>>>> are
> >>>>>>>>>>>>>>> written
> >>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            changelog topic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> thus, there are not
> >>>>>>> conflicts.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Nevertheless, I
> >>> don't
> >>>>> see
> >>>>>> a
> >>>>>>>> big
> >>>>>>>>>> issue
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headers within
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Streams.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> As long as we
> >>> document
> >>>>> it,
> >>>>>>> we
> >>>>>>>>> can
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>> "reserved"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            header keys
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> users are not
> >>> allowed
> >>>>> to
> >>>>>> use
> >>>>>>>>> when
> >>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>> data with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Streams.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> IMHO, this should be
> >>>>> ok.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> I think there is a
> >>> safe
> >>>>>> way
> >>>>>>> to
> >>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>> conflicts,
> >>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > headers
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> only needed in
> >>>>> internal
> >>>>>>>> topics
> >>>>>>>>> (I
> >>>>>>>>>>>>>>> think):
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> For internal and
> >>>>>> changelog
> >>>>>>>>>> topics,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>> namespace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            all headers:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> * user-defined
> >>> headers
> >>>>>> are
> >>>>>>>>>>>> namespaced
> >>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>> "external."
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headerKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> * internal headers
> >>> are
> >>>>>>>>>> namespaced as
> >>>>>>>>>>>>>>>>>>>>>> "internal." +
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headerKey
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> While name spacing
> >>>>> would
> >>>>>> be
> >>>>>>>>>>>> possible,
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> deserialize
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> user headers what
> >>>>> implies
> >>>>>> a
> >>>>>>>>>> runtime
> >>>>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            suggest to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> namespace for now to
> >>>>> avoid
> >>>>>>> the
> >>>>>>>>>>>>>>> overhead.
> >>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > problem in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the future, we can
> >>>>> still
> >>>>>> add
> >>>>>>>>> name
> >>>>>>>>>>>>>>> spacing
> >>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> My main concern
> >>> about
> >>>>> the
> >>>>>>>> design
> >>>>>>>>>> it
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            result KTable:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> If I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> understood the
> >>> proposal
> >>>>>>>>> correctly,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K1,V1>
> >>> table1 =
> >>>>> ...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K2,V2>
> >>> table2 =
> >>>>> ...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K1,V3>
> >>>>> joinedTable
> >>>>>> =
> >>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> implies that the
> >>>>>>> `joinedTable`
> >>>>>>>>> has
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>> as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            left input
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> IMHO, this does not
> >>>>> work
> >>>>>>>> because
> >>>>>>>>>> if
> >>>>>>>>>>>>>>> table2
> >>>>>>>>>>>>>>>>>>>>>> contains
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            multiple rows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> join with a record
> >>> in
> >>>>>> table1
> >>>>>>>>>> (what is
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> main
> >>>>>>>>>>>>>>>>>>>>>> purpose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > foreign
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> join), the result
> >>> table
> >>>>>>> would
> >>>>>>>>> only
> >>>>>>>>>>>>>>>>> contain a
> >>>>>>>>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            join result,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> multiple.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Example:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table1 input stream:
> >>>>> <A,X>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table2 input stream:
> >>>>>>>> <a,(A,1)>,
> >>>>>>>>>>>>>>> <b,(A,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> We use table2 value
> >>> a
> >>>>>>> foreign
> >>>>>>>>> key
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> table1
> >>>>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            "A" joins).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> result key is the
> >>> same
> >>>>> key
> >>>>>>> as
> >>>>>>>>> key
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>> table1,
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            implies that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> result can either be
> >>>>> <A,
> >>>>>>>>>> join(X,1)>
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>> <A,
> >>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            but not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > both.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Because the share
> >>> the
> >>>>> same
> >>>>>>>> key,
> >>>>>>>>>>>>>>> whatever
> >>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we emit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > later,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> overwrite the
> >>> previous
> >>>>>>> result.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> This is the reason
> >>> why
> >>>>> Jan
> >>>>>>>>>> originally
> >>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > combination
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> both primary keys of
> >>>>> the
> >>>>>>> input
> >>>>>>>>>> tables
> >>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            output table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> makes the keys of
> >>> the
> >>>>>> output
> >>>>>>>>> table
> >>>>>>>>>>>>>>> unique
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            store both in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> output table:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Result would be
> >>> <A-a,
> >>>>>>>>> join(X,1)>,
> >>>>>>>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> On 9/4/18 1:36 PM,
> >>> Jan
> >>>>>>>> Filipiak
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Just on remark here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> The high-watermark
> >>>>> could
> >>>>>> be
> >>>>>>>>>>>>>>> disregarded.
> >>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>> decision
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            about the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> depends on the
> >>> size of
> >>>>>> the
> >>>>>>>>>>>> aggregated
> >>>>>>>>>>>>>>>>> map.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> Only 1 element long
> >>>>> maps
> >>>>>>>> would
> >>>>>>>>> be
> >>>>>>>>>>>>>>>>> unpacked
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            forwarded. 0
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > element
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> maps
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> would be published
> >>> as
> >>>>>>> delete.
> >>>>>>>>> Any
> >>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>> count
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> of map entries is
> >>> in
> >>>>>>> "waiting
> >>>>>>>>> for
> >>>>>>>>>>>>>>> correct
> >>>>>>>>>>>>>>>>>>>>>> deletes to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > arrive"-state.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> On 04.09.2018
> >>> 21:29,
> >>>>> Adam
> >>>>>>>>>> Bellemare
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> It does look like I
> >>>>> could
> >>>>>>>>> replace
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            repartition store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> highwater store
> >>> with
> >>>>> a
> >>>>>>>> groupBy
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>> reduce.
> >>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            it looks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> still need to
> >>> store
> >>>>> the
> >>>>>>>>>> highwater
> >>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>>>>> within
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            materialized
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> compare the
> >>> arrival of
> >>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>> (assuming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> THIS is
> >>> correct...).
> >>>>> This
> >>>>>>> in
> >>>>>>>>>> effect
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            design I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> now,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> just with the two
> >>>>> tables
> >>>>>>>> merged
> >>>>>>>>>>>>>>> together.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> -- Guozhang
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

just want to let you guys know that this thing is spiralling out of 
control if you ask me.

First you take away the possibility for the user to optimize. Now you 
pile up complexity to perform some afterwards optimisation, that from my 
POV completely misses the point. As if the actual call to the joiner 
really gonna be an expensive part. It wont. Truth is, you don't have a 
clue which side is gonna be smaller. might be the key you shuffle around 
is >>> than the value on the other side already.

You know my opinion on this. For me its dead, I just leave you the 
message here as an opportunity to reconsider the choices that were made.

Whish y'll a happy new year :)






On 27.12.2018 17:22, Adam Bellemare wrote:
> Hi All
>
> Sorry for the delay - holidays and all. I have since updated the KIP with
> John's original suggestion and have pruned a number of the no longer
> relevant diagrams. Any more comments would be welcomed, otherwise I will
> look to kick off the vote again shortly.
>
> Thanks
> Adam
>
> On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi John and Guozhang
>>
>> Ah yes, I lost that in the mix! Thanks for the convergent solutions - I do
>> think that the attachment that John included makes for a better design. It
>> should also help with overall performance as very high-cardinality foreign
>> keyed data (say millions of events with the same entity) will be able to
>> leverage the multiple nodes for join functionality instead of having it all
>> performed in one node. There is still a bottleneck in the right table
>> having to propagate all those events, but with slimmer structures, less IO
>> and no need to perform the join I think the throughput will be much higher
>> in those scenarios.
>>
>> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
>> version of John's diagram and ensure that the example flow matches
>> correctly. Then I can go back to working on the PR to match the diagram.
>>
>> Thanks both of you for all the help - very much appreciated.
>>
>> Adam
>>
>>
>>
>>
>>
>>
>>
>> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Hi John,
>>>
>>> Just made a pass on your diagram (nice hand-drawing btw!), and obviously
>>> we
>>> are thinking about the same thing :) A neat difference that I like, is
>>> that
>>> in the pre-join repartition topic we can still send message in the format
>>> of `K=k, V=(i=2)` while using "i" as the partition key in
>>> StreamsPartition,
>>> this way we do not need to even augment the key for the repartition topic,
>>> but just do a projection on the foreign key part but trim all other
>>> fields:
>>> as long as we still materialize the store as `A-2` co-located with the
>>> right KTable, that is fine.
>>>
>>> As I mentioned in my previous email, I also think this has a few
>>> advantages
>>> on saving over-the-wire bytes as well as disk bytes.
>>>
>>> Guozhang
>>>
>>>
>>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io> wrote:
>>>
>>>> Hi Guozhang,
>>>>
>>>> Thanks for taking a look! I think Adam's already addressed your
>>> questions
>>>> as well as I could have.
>>>>
>>>> Hi Adam,
>>>>
>>>> Thanks for updating the KIP. It looks great, especially how all the
>>>> need-to-know information is right at the top, followed by the details.
>>>>
>>>> Also, thanks for that high-level diagram. Actually, now that I'm looking
>>>> at it, I think part of my proposal got lost in translation, although I
>>> do
>>>> think that what you have there is also correct.
>>>>
>>>> I sketched up a crude diagram based on yours and attached it to the KIP
>>>> (I'm not sure if attached or inline images work on the mailing list):
>>>>
>>> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
>>>> . It's also attached to this email for convenience.
>>>>
>>>> Hopefully, you can see how it's intended to line up, and which parts are
>>>> modified.
>>>> At a high level, instead of performing the join on the right-hand side,
>>>> we're essentially just registering interest, like "LHS key A wishes to
>>>> receive updates for RHS key 2". Then, when there is a new "interest" or
>>> any
>>>> updates to the RHS records, it "broadcasts" its state back to the LHS
>>>> records who are interested in it.
>>>>
>>>> Thus, instead of sending the LHS values to the RHS joiner workers and
>>> then
>>>> sending the join results back to the LHS worke be co-partitioned and
>>>> validated, we instead only send the LHS *keys* to the RHS workers and
>>> then
>>>> only the RHS k/v back to be joined by the LHS worker.
>>>>
>>>> I've been considering both your diagram and mine, and I *think* what I'm
>>>> proposing has a few advantages.
>>>>
>>>> Here are some points of interest as you look at the diagram:
>>>> * When we extract the foreign key and send it to the Pre-Join
>>> Repartition
>>>> Topic, we can send only the FK/PK pair. There's no need to worry about
>>>> custom partitioner logic, since we can just use the foreign key plainly
>>> as
>>>> the repartition record key. Also, we save on transmitting the LHS value,
>>>> since we only send its key in this step.
>>>> * We also only need to store the RHSKey:LHSKey mapping in the
>>>> MaterializedSubscriptionStore, saving on disk. We can use the same rocks
>>>> key format you proposed and the same algorithm involving range scans
>>> when
>>>> the RHS records get updated.
>>>> * Instead of joining on the right side, all we do is compose a
>>>> re-repartition record so we can broadcast the RHS k/v pair back to the
>>>> original LHS partition. (this is what the "rekey" node is doing)
>>>> * Then, there is a special kind of Joiner that's co-resident in the same
>>>> StreamTask as the LHS table, subscribed to the Post-Join Repartition
>>> Topic.
>>>> ** This Joiner is *not* triggered directly by any changes in the LHS
>>>> KTable. Instead, LHS events indirectly trigger the join via the whole
>>>> lifecycle.
>>>> ** For each event arriving from the Post-Join Repartition Topic, the
>>>> Joiner looks up the corresponding record in the LHS KTable. It validates
>>>> the FK as you noted, discarding any inconsistent events. Otherwise, it
>>>> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the join
>>>> result
>>>> ** Note that the Joiner itself is stateless, so materializing the join
>>>> result is optional, just as with the 1:1 joins.
>>>>
>>>> So in summary:
>>>> * instead of transmitting the LHS keys and values to the right and the
>>>> JoinResult back to the left, we only transmit the LHS keys to the right
>>> and
>>>> the RHS values to the left. Assuming the average RHS value is on smaller
>>>> than or equal to the average join result size, it's a clear win on
>>> broker
>>>> traffic. I think this is actually a reasonable assumption, which we can
>>>> discuss more if you're suspicious.
>>>> * we only need one copy of the data (the left and right tables need to
>>> be
>>>> materialized) and one extra copy of the PK:FK pairs in the Materialized
>>>> Subscription Store. Materializing the join result is optional, just as
>>> with
>>>> the existing 1:1 joins.
>>>> * we still need the fancy range-scan algorithm on the right to locate
>>> all
>>>> interested LHS keys when a RHS value is updated, but we don't need a
>>> custom
>>>> partitioner for either repartition topic (this is of course a
>>> modification
>>>> we could make to your version as well)
>>>>
>>>> How does this sound to you? (And did I miss anything?)
>>>> -John
>>>>
>>>> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
>>> adam.bellemare@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi John & Guozhang
>>>>>
>>>>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the
>>> KIP,
>>>>> pruned much of what I wrote and put a simplified diagram near the top
>>> to
>>>>> illustrate the workflow. I encapsulated Jan's content at the bottom of
>>> the
>>>>> document. I believe it is simpler to read by far now.
>>>>>
>>>>> @Guozhang Wang <wa...@gmail.com>:
>>>>>> #1: rekey left table
>>>>>>    -> source from the left upstream, send to rekey-processor to
>>> generate
>>>>> combined key, and then sink to copartition topic.
>>>>> Correct.
>>>>>
>>>>>> #2: first-join with right table
>>>>>>    -> source from the right table upstream, materialize the right
>>> table.
>>>>>>    -> source from the co-partition topic, materialize the rekeyed left
>>>>> table, join with the right table, rekey back, and then sink to the
>>>>> rekeyed-back topic.
>>>>> Almost - I cleared up the KIP. We do not rekey back yet, as I need the
>>>>> Foreign-Key value generated in #1 above to compare in the resolution
>>>>> stage.
>>>>>
>>>>>> #3: second join
>>>>>>     -> source from the rekeyed-back topic, materialize the rekeyed
>>> back
>>>>> table.
>>>>>>    -> source from the left upstream, materialize the left table, join
>>>>> with
>>>>> the rekeyed back table.
>>>>> Almost - As each event comes in, we just run it through a stateful
>>>>> processor that checks the original ("This") KTable for the key. The
>>> value
>>>>> payload then has the foreignKeyExtractor applied again as in Part #1
>>>>> above,
>>>>> and gets the current foreign key. Then we compare it to the joined
>>> event
>>>>> that we are currently resolving. If they have the same foreign-key,
>>>>> propagate the result out. If they don't, throw the event away.
>>>>>
>>>>> The end result is that we do need to materialize 2 additional tables
>>>>> (left/this-combinedkey table, and the final Joined table) as I've
>>>>> illustrated in the updated KIP. I hope the diagram clears it up a lot
>>>>> better. Please let me know.
>>>>>
>>>>> Thanks again
>>>>> Adam
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>>>
>>>>>> John,
>>>>>>
>>>>>> Thanks a lot for the suggestions on refactoring the wiki, I agree
>>> with
>>>>> you
>>>>>> that we should consider the KIP proposal to be easily understood by
>>>>> anyone
>>>>>> in the future to read, and hence should provide a good summary on the
>>>>>> user-facing interfaces, as well as rejected alternatives to represent
>>>>>> briefly "how we came a long way to this conclusion, and what we have
>>>>>> argued, disagreed, and agreed about, etc" so that readers do not
>>> need to
>>>>>> dig into the DISCUSS thread to get all the details. We can, of
>>> course,
>>>>> keep
>>>>>> the implementation details like "workflows" on the wiki page as a
>>>>> addendum
>>>>>> section since it also has correlations.
>>>>>>
>>>>>> Regarding your proposal on comment 6): that's a very interesting
>>> idea!
>>>>> Just
>>>>>> to clarify that I understands it fully correctly: the proposal's
>>>>> resulted
>>>>>> topology is still the same as the current proposal, where we will
>>> have 3
>>>>>> sub-topologies for this operator:
>>>>>>
>>>>>> #1: rekey left table
>>>>>>     -> source from the left upstream, send to rekey-processor to
>>> generate
>>>>>> combined key, and then sink to copartition topic.
>>>>>>
>>>>>> #2: first-join with right table
>>>>>>     -> source from the right table upstream, materialize the right
>>> table.
>>>>>>     -> source from the co-partition topic, materialize the rekeyed
>>> left
>>>>>> table, join with the right table, rekey back, and then sink to the
>>>>>> rekeyed-back topic.
>>>>>>
>>>>>> #3: second join
>>>>>>     -> source from the rekeyed-back topic, materialize the rekeyed
>>> back
>>>>>> table.
>>>>>>     -> source from the left upstream, materialize the left table, join
>>>>> with
>>>>>> the rekeyed back table.
>>>>>>
>>>>>> Sub-topology #1 and #3 may be merged to a single sub-topology since
>>>>> both of
>>>>>> them read from the left table source stream. In this workflow, we
>>> need
>>>>> to
>>>>>> materialize 4 tables (left table in #3, right table in #2, rekeyed
>>> left
>>>>>> table in #2, rekeyed-back table in #3), and 2 repartition topics
>>>>>> (copartition topic, rekeyed-back topic).
>>>>>>
>>>>>> Compared with Adam's current proposal in the workflow overview, it
>>> has
>>>>> the
>>>>>> same num.materialize tables (left table, rekeyed left table, right
>>>>> table,
>>>>>> out-of-ordering resolver table), and same num.internal topics (two).
>>> The
>>>>>> advantage is that on the copartition topic, we can save bandwidth by
>>> not
>>>>>> sending value, and in #2 the rekeyed left table is smaller since we
>>> do
>>>>> not
>>>>>> have any values to materialize. Is that right?
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
>>> wrote:
>>>>>>
>>>>>>> Hi Adam,
>>>>>>>
>>>>>>> Given that the committers are all pretty busy right now, I think
>>> that
>>>>> it
>>>>>>> would help if you were to refactor the KIP a little to reduce the
>>>>>> workload
>>>>>>> for reviewers.
>>>>>>>
>>>>>>> I'd recommend the following changes:
>>>>>>> * relocate all internal details to a section at the end called
>>>>> something
>>>>>>> like "Implementation Notes" or something like that.
>>>>>>> * rewrite the rest of the KIP to be a succinct as possible and
>>> mention
>>>>>> only
>>>>>>> publicly-facing API changes.
>>>>>>> ** for example, the interface that you've already listed there, as
>>>>> well
>>>>>> as
>>>>>>> a textual description of the guarantees we'll be providing (join
>>>>> result
>>>>>> is
>>>>>>> copartitioned with the LHS, and the join result is guaranteed
>>> correct)
>>>>>>>
>>>>>>> A good target would be that the whole main body of the KIP,
>>> including
>>>>>>> Status, Motivation, Proposal, Justification, and Rejected
>>> Alternatives
>>>>>> all
>>>>>>> fit "above the fold" (i.e., all fit on the screen at a comfortable
>>>>> zoom
>>>>>>> level).
>>>>>>> I think the only real Rejected Alternative that bears mention at
>>> this
>>>>>> point
>>>>>>> is KScatteredTable, which you could just include the executive
>>>>> summary on
>>>>>>> (no implementation details), and link to extra details in the
>>>>>>> Implementation Notes section.
>>>>>>>
>>>>>>> Taking a look at the wiki page, ~90% of the text there is internal
>>>>>> detail,
>>>>>>> which is useful for the dubious, but doesn't need to be ratified
>>> in a
>>>>>> vote
>>>>>>> (and would be subject to change without notice in the future
>>> anyway).
>>>>>>> There's also a lot of conflicting discussion, as you've very
>>>>> respectfully
>>>>>>> tried to preserve the original proposal from Jan while adding your
>>>>> own.
>>>>>>> Isolating all this information in a dedicated section at the bottom
>>>>> frees
>>>>>>> the voters up to focus on the public API part of the proposal,
>>> which
>>>>> is
>>>>>>> really all they need to consider.
>>>>>>>
>>>>>>> Plus, it'll be clear to future readers which parts of the document
>>> are
>>>>>>> enduring, and which parts are a snapshot of our implementation
>>>>> thinking
>>>>>> at
>>>>>>> the time.
>>>>>>>
>>>>>>> I'm suggesting this because I suspect that the others haven't made
>>>>> time
>>>>>> to
>>>>>>> review it partly because it seems daunting. If it seems like it
>>> would
>>>>> be
>>>>>> a
>>>>>>> huge time investment to review, people will just keep putting it
>>> off.
>>>>> But
>>>>>>> if the KIP is a single page, then they'll be more inclined to give
>>> it
>>>>> a
>>>>>>> read.
>>>>>>>
>>>>>>> Honestly, I don't think the KIP itself is that controversial (apart
>>>>> from
>>>>>>> the scattered table thing (sorry, Jan) ). Most of the discussion
>>> has
>>>>> been
>>>>>>> around the implementation, which we can continue more effectively
>>> in
>>>>> a PR
>>>>>>> once the KIP has passed.
>>>>>>>
>>>>>>> How does that sound?
>>>>>>> -John
>>>>>>>
>>>>>>> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
>>>>> adam.bellemare@gmail.com
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> 1) I believe that the resolution mechanism John has proposed is
>>>>>>> sufficient
>>>>>>>> - it is clean and easy and doesn't require additional RocksDB
>>>>> stores,
>>>>>>> which
>>>>>>>> reduces the footprint greatly. I don't think we need to resolve
>>>>> based
>>>>>> on
>>>>>>>> timestamp or offset anymore, but if we decide to do to that
>>> would be
>>>>>>> within
>>>>>>>> the bounds of the existing API.
>>>>>>>>
>>>>>>>> 2) Is the current API sufficient, or does it need to be altered
>>> to
>>>>> go
>>>>>>> back
>>>>>>>> to vote?
>>>>>>>>
>>>>>>>> 3) KScatteredTable implementation can always be added in a future
>>>>>>> revision.
>>>>>>>> This API does not rule it out. This implementation of this
>>> function
>>>>>> would
>>>>>>>> simply be replaced with `KScatteredTable.resolve()` while still
>>>>>>> maintaining
>>>>>>>> the existing API, thereby giving both features as Jan outlined
>>>>> earlier.
>>>>>>>> Would this work?
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks Guozhang, John and Jan
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <john@confluent.io
>>>>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi, all,
>>>>>>>>>
>>>>>>>>>>> In fact, we
>>>>>>>>>>> can just keep a single final-result store with timestamps
>>> and
>>>>>> reject
>>>>>>>>> values
>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>
>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>> offset
>>>>>>> of
>>>>>>>>>> the original message.
>>>>>>>>>
>>>>>>>>> Thanks for this point, Jan.
>>>>>>>>>
>>>>>>>>> KIP-258 is merely to allow embedding the record timestamp  in
>>> the
>>>>> k/v
>>>>>>>>> store,
>>>>>>>>> as well as providing a storage-format upgrade path.
>>>>>>>>>
>>>>>>>>> I might have missed it, but I think we have yet to discuss
>>> whether
>>>>>> it's
>>>>>>>>> safe
>>>>>>>>> or desirable just to swap topic-ordering our for
>>>>> timestamp-ordering.
>>>>>>> This
>>>>>>>>> is
>>>>>>>>> a very deep topic, and I think it would only pollute the
>>> current
>>>>>>>>> discussion.
>>>>>>>>>
>>>>>>>>> What Adam has proposed is safe, given the *current* ordering
>>>>>> semantics
>>>>>>>>> of the system. If we can agree on his proposal, I think we can
>>>>> merge
>>>>>>> the
>>>>>>>>> feature well before the conversation about timestamp ordering
>>> even
>>>>>>> takes
>>>>>>>>> place, much less reaches a conclusion. In the mean time, it
>>> would
>>>>>> seem
>>>>>>> to
>>>>>>>>> be unfortunate to have one join operator with different
>>> ordering
>>>>>>>> semantics
>>>>>>>>> from every other KTable operator.
>>>>>>>>>
>>>>>>>>> If and when that timestamp discussion takes place, many (all?)
>>>>> KTable
>>>>>>>>> operations
>>>>>>>>> will need to be updated, rendering the many:one join a small
>>>>> marginal
>>>>>>>> cost.
>>>>>>>>>
>>>>>>>>> And, just to plug it again, I proposed an algorithm above that
>>> I
>>>>>>> believe
>>>>>>>>> provides
>>>>>>>>> correct ordering without any additional metadata, and
>>> regardless
>>>>> of
>>>>>> the
>>>>>>>>> ordering semantics. I didn't bring it up further, because I
>>> felt
>>>>> the
>>>>>>> KIP
>>>>>>>>> only needs
>>>>>>>>> to agree on the public API, and we can discuss the
>>> implementation
>>>>> at
>>>>>>>>> leisure in
>>>>>>>>> a PR...
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 10.12.2018 07:42, Guozhang Wang wrote:
>>>>>>>>>>> Hello Adam / Jan / John,
>>>>>>>>>>>
>>>>>>>>>>> Sorry for being late on this thread! I've finally got some
>>>>> time
>>>>>>> this
>>>>>>>>>>> weekend to cleanup a load of tasks on my queue (actually
>>> I've
>>>>>> also
>>>>>>>>>> realized
>>>>>>>>>>> there are a bunch of other things I need to enqueue while
>>>>>> cleaning
>>>>>>>> them
>>>>>>>>>> up
>>>>>>>>>>> --- sth I need to improve on my side). So here are my
>>>>> thoughts:
>>>>>>>>>>>
>>>>>>>>>>> Regarding the APIs: I like the current written API in the
>>> KIP.
>>>>>> More
>>>>>>>>>>> generally I'd prefer to keep the 1) one-to-many join
>>>>>>> functionalities
>>>>>>>> as
>>>>>>>>>>> well as 2) other join types than inner as separate KIPs
>>> since
>>>>> 1)
>>>>>>> may
>>>>>>>>>> worth
>>>>>>>>>>> a general API refactoring that can benefit not only
>>> foreignkey
>>>>>>> joins
>>>>>>>>> but
>>>>>>>>>>> collocate joins as well (e.g. an extended proposal of
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
>>>>>>>>>> ),
>>>>>>>>>>> and I'm not sure if other join types would actually be
>>> needed
>>>>>>> (maybe
>>>>>>>>> left
>>>>>>>>>>> join still makes sense), so it's better to
>>>>>>>>> wait-for-people-to-ask-and-add
>>>>>>>>>>> than add-sth-that-no-one-uses.
>>>>>>>>>>>
>>>>>>>>>>> Regarding whether we enforce step 3) / 4) v.s. introducing
>>> a
>>>>>>>>>>> KScatteredTable for users to inject their own optimization:
>>>>> I'd
>>>>>>>> prefer
>>>>>>>>> to
>>>>>>>>>>> do the current option as-is, and my main rationale is for
>>>>>>>> optimization
>>>>>>>>>>> rooms inside the Streams internals and the API
>>> succinctness.
>>>>> For
>>>>>>>>> advanced
>>>>>>>>>>> users who may indeed prefer KScatteredTable and do their
>>> own
>>>>>>>>>> optimization,
>>>>>>>>>>> while it is too much of the work to use Processor API
>>>>> directly, I
>>>>>>>> think
>>>>>>>>>> we
>>>>>>>>>>> can still extend the current API to support it in the
>>> future
>>>>> if
>>>>>> it
>>>>>>>>>> becomes
>>>>>>>>>>> necessary.
>>>>>>>>>>
>>>>>>>>>> no internal optimization potential. it's a myth
>>>>>>>>>>
>>>>>>>>>> ¯\_(ツ)_/¯
>>>>>>>>>>
>>>>>>>>>> :-)
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Another note about step 4) resolving out-of-ordering data,
>>> as
>>>>> I
>>>>>>>>> mentioned
>>>>>>>>>>> before I think with KIP-258 (embedded timestamp with
>>> key-value
>>>>>>> store)
>>>>>>>>> we
>>>>>>>>>>> can actually make this step simpler than the current
>>>>> proposal. In
>>>>>>>> fact,
>>>>>>>>>> we
>>>>>>>>>>> can just keep a single final-result store with timestamps
>>> and
>>>>>>> reject
>>>>>>>>>> values
>>>>>>>>>>> that have a smaller timestamp, is that right?
>>>>>>>>>>
>>>>>>>>>> Which is the correct output should at least be decided on the
>>>>>> offset
>>>>>>> of
>>>>>>>>>> the original message.
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> That's all I have in mind now. Again, great appreciation to
>>>>> Adam
>>>>>> to
>>>>>>>>> make
>>>>>>>>>>> such HUGE progress on this KIP!
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> If they don't find the time:
>>>>>>>>>>>> They usually take the opposite path from me :D
>>>>>>>>>>>> so the answer would be clear.
>>>>>>>>>>>>
>>>>>>>>>>>> hence my suggestion to vote.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 04.12.2018 21:06, Adam Bellemare wrote:
>>>>>>>>>>>>> Hi Guozhang and Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> I know both of you are quite busy, but we've gotten this
>>> KIP
>>>>>> to a
>>>>>>>>> point
>>>>>>>>>>>>> where we need more guidance on the API (perhaps a bit of
>>> a
>>>>>>>>> tie-breaker,
>>>>>>>>>>>> if
>>>>>>>>>>>>> you will). If you have anyone else you may think should
>>>>> look at
>>>>>>>> this,
>>>>>>>>>>>>> please tag them accordingly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> The scenario is as such:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Current Option:
>>>>>>>>>>>>> API:
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>>>>>>>>>>>>> 1) Rekey the data to CombinedKey, and shuffles it to the
>>>>>>> partition
>>>>>>>>> with
>>>>>>>>>>>> the
>>>>>>>>>>>>> foreignKey (repartition 1)
>>>>>>>>>>>>> 2) Join the data
>>>>>>>>>>>>> 3) Shuffle the data back to the original node
>>> (repartition
>>>>> 2)
>>>>>>>>>>>>> 4) Resolve out-of-order arrival / race condition due to
>>>>>>> foreign-key
>>>>>>>>>>>> changes.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Alternate Option:
>>>>>>>>>>>>> Perform #1 and #2 above, and return a KScatteredTable.
>>>>>>>>>>>>> - It would be keyed on a wrapped key function:
>>>>> <CombinedKey<KO,
>>>>>>> K>,
>>>>>>>>> VR>
>>>>>>>>>>>> (KO
>>>>>>>>>>>>> = Other Table Key, K = This Table Key, VR = Joined
>>> Result)
>>>>>>>>>>>>> - KScatteredTable.resolve() would perform #3 and #4 but
>>>>>>> otherwise a
>>>>>>>>>> user
>>>>>>>>>>>>> would be able to perform additional functions directly
>>> from
>>>>> the
>>>>>>>>>>>>> KScatteredTable (TBD - currently out of scope).
>>>>>>>>>>>>> - John's analysis 2-emails up is accurate as to the
>>>>> tradeoffs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Current Option is coded as-is. Alternate option is
>>> possible,
>>>>>> but
>>>>>>>> will
>>>>>>>>>>>>> require for implementation details to be made in the API
>>> and
>>>>>> some
>>>>>>>>>>>> exposure
>>>>>>>>>>>>> of new data structures into the API (ie: CombinedKey).
>>>>>>>>>>>>>
>>>>>>>>>>>>> I appreciate any insight into this.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for your feedback and assistance. I think your
>>>>> summary
>>>>>> is
>>>>>>>>>>>> accurate
>>>>>>>>>>>>>> from my perspective. Additionally, I would like to add
>>> that
>>>>>>> there
>>>>>>>>> is a
>>>>>>>>>>>> risk
>>>>>>>>>>>>>> of inconsistent final states without performing the
>>>>>> resolution.
>>>>>>>> This
>>>>>>>>>> is
>>>>>>>>>>>> a
>>>>>>>>>>>>>> major concern for me as most of the data I have dealt
>>> with
>>>>> is
>>>>>>>>> produced
>>>>>>>>>>>> by
>>>>>>>>>>>>>> relational databases. We have seen a number of cases
>>> where
>>>>> a
>>>>>>> user
>>>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>>> Rails UI has modified the field (foreign key), realized
>>>>> they
>>>>>>> made
>>>>>>>> a
>>>>>>>>>>>>>> mistake, and then updated the field again with a new
>>> key.
>>>>> The
>>>>>>>> events
>>>>>>>>>> are
>>>>>>>>>>>>>> propagated out as they are produced, and as such we have
>>>>> had
>>>>>>>>>> real-world
>>>>>>>>>>>>>> cases where these inconsistencies were propagated
>>>>> downstream
>>>>>> as
>>>>>>>> the
>>>>>>>>>>>> final
>>>>>>>>>>>>>> values due to the race conditions in the fanout of the
>>>>> data.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This solution that I propose values correctness of the
>>>>> final
>>>>>>>> result
>>>>>>>>>> over
>>>>>>>>>>>>>> other factors.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> We could always move this function over to using a
>>>>>>> KScatteredTable
>>>>>>>>>>>>>> implementation in the future, and simply deprecate it
>>> this
>>>>>> join
>>>>>>>> API
>>>>>>>>> in
>>>>>>>>>>>>>> time. I think I would like to hear more from some of the
>>>>> other
>>>>>>>> major
>>>>>>>>>>>>>> committers on which course of action they would think is
>>>>> best
>>>>>>>> before
>>>>>>>>>> any
>>>>>>>>>>>>>> more coding is done.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks again
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
>>>>>> john@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Jan and Adam,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Wow, thanks for doing that test, Adam. Those results
>>> are
>>>>>>>>> encouraging.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for your performance experience as well, Jan. I
>>>>> agree
>>>>>>> that
>>>>>>>>>>>> avoiding
>>>>>>>>>>>>>>> unnecessary join outputs is especially important when
>>> the
>>>>>>> fan-out
>>>>>>>>> is
>>>>>>>>>> so
>>>>>>>>>>>>>>> high. I suppose this could also be built into the
>>>>>>> implementation
>>>>>>>>>> we're
>>>>>>>>>>>>>>> discussing, but it wouldn't have to be specified in the
>>>>> KIP
>>>>>>>> (since
>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>> API-transparent optimization).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As far as whether or not to re-repartition the data, I
>>>>> didn't
>>>>>>>> bring
>>>>>>>>>> it
>>>>>>>>>>>> up
>>>>>>>>>>>>>>> because it sounded like the two of you agreed to leave
>>> the
>>>>>> KIP
>>>>>>>>> as-is,
>>>>>>>>>>>>>>> despite the disagreement.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> If you want my opinion, I feel like both approaches are
>>>>>>>> reasonable.
>>>>>>>>>>>>>>> It sounds like Jan values more the potential for
>>>>> developers
>>>>>> to
>>>>>>>>>> optimize
>>>>>>>>>>>>>>> their topologies to re-use the intermediate nodes,
>>> whereas
>>>>>> Adam
>>>>>>>>>> places
>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>> value on having a single operator that people can use
>>>>> without
>>>>>>>> extra
>>>>>>>>>>>> steps
>>>>>>>>>>>>>>> at the end.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Personally, although I do find it exceptionally
>>> annoying
>>>>>> when a
>>>>>>>>>>>> framework
>>>>>>>>>>>>>>> gets in my way when I'm trying to optimize something,
>>> it
>>>>>> seems
>>>>>>>>> better
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>> for a single operation.
>>>>>>>>>>>>>>> * Encapsulating the internal transitions gives us
>>>>> significant
>>>>>>>>>> latitude
>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the implementation (for example, joining only at the
>>> end,
>>>>> not
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> middle
>>>>>>>>>>>>>>> to avoid extra data copying and out-of-order
>>> resolution;
>>>>> how
>>>>>> we
>>>>>>>>>>>> represent
>>>>>>>>>>>>>>> the first repartition keys (combined keys vs. value
>>>>> vectors),
>>>>>>>>> etc.).
>>>>>>>>>>>> If we
>>>>>>>>>>>>>>> publish something like a KScatteredTable with the
>>>>>>>> right-partitioned
>>>>>>>>>>>> joined
>>>>>>>>>>>>>>> data, then the API pretty much locks in the
>>>>> implementation as
>>>>>>>> well.
>>>>>>>>>>>>>>> * The API seems simpler to understand and use. I do
>>> mean
>>>>>>> "seems";
>>>>>>>>> if
>>>>>>>>>>>>>>> anyone
>>>>>>>>>>>>>>> wants to make the case that KScatteredTable is actually
>>>>>>> simpler,
>>>>>>>> I
>>>>>>>>>>>> think
>>>>>>>>>>>>>>> hypothetical usage code would help. From a relational
>>>>> algebra
>>>>>>>>>>>> perspective,
>>>>>>>>>>>>>>> it seems like KTable.join(KTable) should produce a new
>>>>> KTable
>>>>>>> in
>>>>>>>>> all
>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>> * That said, there might still be room in the API for a
>>>>>>> different
>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>> like what Jan has proposed to scatter a KTable, and
>>> then
>>>>> do
>>>>>>>> things
>>>>>>>>>> like
>>>>>>>>>>>>>>> join, re-group, etc from there... I'm not sure; I
>>> haven't
>>>>>>> thought
>>>>>>>>>>>> through
>>>>>>>>>>>>>>> all the consequences yet.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This is all just my opinion after thinking over the
>>>>>> discussion
>>>>>>> so
>>>>>>>>>>>> far...
>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Updated the PR to take into account John's feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I did some preliminary testing for the performance of
>>> the
>>>>>>>>>> prefixScan.
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> have attached the file, but I will also include the
>>> text
>>>>> in
>>>>>>> the
>>>>>>>>> body
>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>> for archival purposes (I am not sure what happens to
>>>>>> attached
>>>>>>>>>> files).
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> also updated the PR and the KIP accordingly.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Summary: It scales exceptionally well for scanning
>>> large
>>>>>>> values
>>>>>>>> of
>>>>>>>>>>>>>>>> records. As Jan mentioned previously, the real issue
>>>>> would
>>>>>> be
>>>>>>>> more
>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>> processing the resulting records after obtaining them.
>>>>> For
>>>>>>>>> instance,
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> takes approximately ~80-120 mS to flush the buffer
>>> and a
>>>>>>> further
>>>>>>>>>>>>>>> ~35-85mS
>>>>>>>>>>>>>>>> to scan 27.5M records, obtaining matches for 2.5M of
>>>>> them.
>>>>>>>>> Iterating
>>>>>>>>>>>>>>>> through the records just to generate a simple count
>>>>> takes ~
>>>>>> 40
>>>>>>>>> times
>>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>>> than the flush + scan combined.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> ============================================================================================
>>>>>>>>>>>>>>>> Setup:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> ============================================================================================
>>>>>>>>>>>>>>>> Java 9 with default settings aside from a 512 MB heap
>>>>>>> (Xmx512m,
>>>>>>>>>>>> Xms512m)
>>>>>>>>>>>>>>>> CPU: i7 2.2 Ghz.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Note: I am using a slightly-modified,
>>> directly-accessible
>>>>>>> Kafka
>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>> implementation (RocksDB.java, basically just avoiding
>>> the
>>>>>>>>>>>>>>>> ProcessorContext).
>>>>>>>>>>>>>>>> There are no modifications to the default RocksDB
>>> values
>>>>>>>> provided
>>>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 2.1/trunk release.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> keysize = 128 bytes
>>>>>>>>>>>>>>>> valsize = 512 bytes
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Step 1:
>>>>>>>>>>>>>>>> Write X positive matching events: (key = prefix +
>>>>>> left-padded
>>>>>>>>>>>>>>>> auto-incrementing integer)
>>>>>>>>>>>>>>>> Step 2:
>>>>>>>>>>>>>>>> Write 10X negative matching events (key = left-padded
>>>>>>>>>>>> auto-incrementing
>>>>>>>>>>>>>>>> integer)
>>>>>>>>>>>>>>>> Step 3:
>>>>>>>>>>>>>>>> Perform flush
>>>>>>>>>>>>>>>> Step 4:
>>>>>>>>>>>>>>>> Perform prefixScan
>>>>>>>>>>>>>>>> Step 5:
>>>>>>>>>>>>>>>> Iterate through return Iterator and validate the
>>> count of
>>>>>>>> expected
>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> ============================================================================================
>>>>>>>>>>>>>>>> Results:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> ============================================================================================
>>>>>>>>>>>>>>>> X = 1k (11k events total)
>>>>>>>>>>>>>>>> Flush Time = 39 mS
>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>> 6.9 MB disk
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>> X = 10k (110k events total)
>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>> Scan Time = 8 mS
>>>>>>>>>>>>>>>> 127 MB
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>> X = 100k (1.1M events total)
>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>> Flush Time = 60 mS
>>>>>>>>>>>>>>>> Scan Time = 12 mS
>>>>>>>>>>>>>>>> 678 MB
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>> Flush Time = 45 mS
>>>>>>>>>>>>>>>> Scan Time = 7 mS
>>>>>>>>>>>>>>>> 576 MB
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>> X = 1MB (11M events total)
>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>> Flush Time = 52 mS
>>>>>>>>>>>>>>>> Scan Time = 19 mS
>>>>>>>>>>>>>>>> 7.2 GB
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>> Flush Time = 84 mS
>>>>>>>>>>>>>>>> Scan Time = 34 mS
>>>>>>>>>>>>>>>> 9.1 GB
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>> X = 2.5M (27.5M events total)
>>>>>>>>>>>>>>>> Test1:
>>>>>>>>>>>>>>>> Flush Time = 82 mS
>>>>>>>>>>>>>>>> Scan Time = 63 mS
>>>>>>>>>>>>>>>> 17GB - 276 sst files
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Test2:
>>>>>>>>>>>>>>>> Flush Time = 116 mS
>>>>>>>>>>>>>>>> Scan Time = 35 mS
>>>>>>>>>>>>>>>> 23GB - 361 sst files
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Test3:
>>>>>>>>>>>>>>>> Flush Time = 103 mS
>>>>>>>>>>>>>>>> Scan Time = 82 mS
>>>>>>>>>>>>>>>> 19 GB - 300 sst files
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> --------------------------------------------------------------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I had to limit my testing on my laptop to X = 2.5M
>>>>> events. I
>>>>>>>> tried
>>>>>>>>>> to
>>>>>>>>>>>> go
>>>>>>>>>>>>>>>> to X = 10M (110M events) but RocksDB was going into
>>> the
>>>>>> 100GB+
>>>>>>>>> range
>>>>>>>>>>>>>>> and my
>>>>>>>>>>>>>>>> laptop ran out of disk. More extensive testing could
>>> be
>>>>> done
>>>>>>>> but I
>>>>>>>>>>>>>>> suspect
>>>>>>>>>>>>>>>> that it would be in line with what we're seeing in the
>>>>>> results
>>>>>>>>>> above.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> At this point in time, I think the only major
>>> discussion
>>>>>> point
>>>>>>>> is
>>>>>>>>>>>> really
>>>>>>>>>>>>>>>> around what Jan and I have disagreed on:
>>> repartitioning
>>>>>> back +
>>>>>>>>>>>> resolving
>>>>>>>>>>>>>>>> potential out of order issues or leaving that up to
>>> the
>>>>>> client
>>>>>>>> to
>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks folks,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Sorry that this discussion petered out... I think
>>> the
>>>>> 2.1
>>>>>>>>> release
>>>>>>>>>>>>>>>>> caused an
>>>>>>>>>>>>>>>>>> extended distraction that pushed it off everyone's
>>>>> radar
>>>>>>>> (which
>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> precisely Adam's concern). Personally, I've also had
>>>>> some
>>>>>>>> extend
>>>>>>>>>>>>>>>>>> distractions of my own that kept (and continue to
>>>>> keep) me
>>>>>>>>>>>>>>> preoccupied.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> However, calling for a vote did wake me up, so I
>>> guess
>>>>> Jan
>>>>>>> was
>>>>>>>>> on
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>> track!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've gone back and reviewed the whole KIP document
>>> and
>>>>> the
>>>>>>>> prior
>>>>>>>>>>>>>>>>>> discussion, and I'd like to offer a few thoughts:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> API Thoughts:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1. If I read the KIP right, you are proposing a
>>>>>> many-to-one
>>>>>>>>> join.
>>>>>>>>>>>>>>> Could
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
>>>>> flip
>>>>>>> the
>>>>>>>>>> design
>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>> and make it a oneToManyJoin?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The proposed name "joinOnForeignKey" disguises the
>>> join
>>>>>>> type,
>>>>>>>>> and
>>>>>>>>>> it
>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>> like it might trick some people into using it for a
>>>>>>> one-to-one
>>>>>>>>>> join.
>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>> would work, of course, but it would be super
>>>>> inefficient
>>>>>>>>> compared
>>>>>>>>>> to
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> simple rekey-and-join.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2. I might have missed it, but I don't think it's
>>>>>> specified
>>>>>>>>>> whether
>>>>>>>>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>>> inner, outer, or left join. I'm guessing an outer
>>>>> join, as
>>>>>>>>>>>>>>> (neglecting
>>>>>>>>>>>>>>>>> IQ),
>>>>>>>>>>>>>>>>>> the rest can be achieved by filtering or by handling
>>>>> it in
>>>>>>> the
>>>>>>>>>>>>>>>>> ValueJoiner.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
>>> quite
>>>>>>> right.
>>>>>>>>>>>>>>>>>> 3a. Regarding Serialized: There are a few different
>>>>>>> paradigms
>>>>>>>> in
>>>>>>>>>>>>>>> play in
>>>>>>>>>>>>>>>>>> the Streams API, so it's confusing, but instead of
>>>>> three
>>>>>>>>>> Serialized
>>>>>>>>>>>>>>>>> args, I
>>>>>>>>>>>>>>>>>> think it would be better to have one that allows
>>>>>>> (optionally)
>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>> the 4
>>>>>>>>>>>>>>>>>> incoming serdes. The result serde is defined by the
>>>>>>>>> Materialized.
>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> incoming serdes can be optional because they might
>>>>> already
>>>>>>> be
>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> the source KTables, or the default serdes from the
>>>>> config
>>>>>>>> might
>>>>>>>>> be
>>>>>>>>>>>>>>>>>> applicable.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3b. Is the StreamPartitioner necessary? The other
>>> joins
>>>>>>> don't
>>>>>>>>>> allow
>>>>>>>>>>>>>>>>> setting
>>>>>>>>>>>>>>>>>> one, and it seems like it might actually be harmful,
>>>>> since
>>>>>>> the
>>>>>>>>>> rekey
>>>>>>>>>>>>>>>>>> operation needs to produce results that are
>>>>> co-partitioned
>>>>>>>> with
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> "other"
>>>>>>>>>>>>>>>>>> KTable.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 4. I'm fine with the "reserved word" header, but I
>>>>> didn't
>>>>>>>>> actually
>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>> what Matthias meant about namespacing requiring
>>>>>>>> "deserializing"
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>> header. The headers are already Strings, so I don't
>>>>> think
>>>>>>> that
>>>>>>>>>>>>>>>>>> deserialization is required. If we applied the
>>>>> namespace
>>>>>> at
>>>>>>>>> source
>>>>>>>>>>>>>>> nodes
>>>>>>>>>>>>>>>>>> and stripped it at sink nodes, this would be
>>>>> practically
>>>>>> no
>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> advantage of the namespace idea is that no public
>>> API
>>>>>> change
>>>>>>>> wrt
>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>> needs to happen, and no restrictions need to be
>>> placed
>>>>> on
>>>>>>>> users'
>>>>>>>>>>>>>>>>> headers.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> (Although I'm wondering if we can get away without
>>> the
>>>>>>> header
>>>>>>>> at
>>>>>>>>>>>>>>> all...
>>>>>>>>>>>>>>>>>> stay tuned)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 5. I also didn't follow the discussion about the HWM
>>>>> table
>>>>>>>>> growing
>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>> bound. As I read it, the HWM table is effectively
>>>>>>> implementing
>>>>>>>>> OCC
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> resolve the problem you noted with disordering when
>>> the
>>>>>>> rekey
>>>>>>>> is
>>>>>>>>>>>>>>>>>> reversed... particularly notable when the FK
>>> changes.
>>>>> As
>>>>>>> such,
>>>>>>>>> it
>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> needs to track the most recent "version" (the
>>> offset in
>>>>>> the
>>>>>>>>> source
>>>>>>>>>>>>>>>>>> partition) of each key. Therefore, it should have
>>> the
>>>>> same
>>>>>>>>> number
>>>>>>>>>> of
>>>>>>>>>>>>>>>>> keys
>>>>>>>>>>>>>>>>>> as the source table at all times.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I see that you are aware of KIP-258, which I think
>>>>> might
>>>>>> be
>>>>>>>>>> relevant
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> couple of ways. One: it's just about storing the
>>>>> timestamp
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> store, but the ultimate idea is to effectively use
>>> the
>>>>>>>> timestamp
>>>>>>>>>> as
>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>> OCC
>>>>>>>>>>>>>>>>>> "version" to drop disordered updates. You wouldn't
>>>>> want to
>>>>>>> use
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> timestamp for this operation, but if you were to
>>> use a
>>>>>>> similar
>>>>>>>>>>>>>>>>> mechanism to
>>>>>>>>>>>>>>>>>> store the source offset in the store alongside the
>>>>>> re-keyed
>>>>>>>>>> values,
>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>> you could avoid a separate table.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 6. You and Jan have been thinking about this for a
>>> long
>>>>>>> time,
>>>>>>>> so
>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>> probably missed something here, but I'm wondering
>>> if we
>>>>>> can
>>>>>>>>> avoid
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> HWM
>>>>>>>>>>>>>>>>>> tracking at all and resolve out-of-order during a
>>> final
>>>>>> join
>>>>>>>>>>>>>>> instead...
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Let's say we're joining a left table (Integer K:
>>> Letter
>>>>>> FK,
>>>>>>>>> (other
>>>>>>>>>>>>>>>>> data))
>>>>>>>>>>>>>>>>>> to a right table (Letter K: (some data)).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>> 1: (A, xyz)
>>>>>>>>>>>>>>>>>> 2: (B, asd)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Right table:
>>>>>>>>>>>>>>>>>> A: EntityA
>>>>>>>>>>>>>>>>>> B: EntityB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We could do a rekey as you proposed with a combined
>>>>> key,
>>>>>> but
>>>>>>>> not
>>>>>>>>>>>>>>>>>> propagating the value at all..
>>>>>>>>>>>>>>>>>> Rekey table:
>>>>>>>>>>>>>>>>>> A-1: (dummy value)
>>>>>>>>>>>>>>>>>> B-2: (dummy value)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Which we then join with the right table to produce:
>>>>>>>>>>>>>>>>>> A-1: EntityA
>>>>>>>>>>>>>>>>>> B-2: EntityB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Which gets rekeyed back:
>>>>>>>>>>>>>>>>>> 1: A, EntityA
>>>>>>>>>>>>>>>>>> 2: B, EntityB
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> And finally we do the actual join:
>>>>>>>>>>>>>>>>>> Result table:
>>>>>>>>>>>>>>>>>> 1: ((A, xyz), EntityA)
>>>>>>>>>>>>>>>>>> 2: ((B, asd), EntityB)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The thing is that in that last join, we have the
>>>>>> opportunity
>>>>>>>> to
>>>>>>>>>>>>>>> compare
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> current FK in the left table with the incoming PK of
>>>>> the
>>>>>>> right
>>>>>>>>>>>>>>> table. If
>>>>>>>>>>>>>>>>>> they don't match, we just drop the event, since it
>>>>> must be
>>>>>>>>>> outdated.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> In your KIP, you gave an example in which (1: A,
>>> xyz)
>>>>> gets
>>>>>>>>> updated
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> (1:
>>>>>>>>>>>>>>>>>> B, xyz), ultimately yielding a conundrum about
>>> whether
>>>>> the
>>>>>>>> final
>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> should be (1: null) or (1: joined-on-B). With the
>>>>>> algorithm
>>>>>>>>> above,
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
>>>>> (B,
>>>>>>> xyz),
>>>>>>>>> (B,
>>>>>>>>>>>>>>>>>> EntityB)). It seems like this does give you enough
>>>>>>> information
>>>>>>>>> to
>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> right choice, regardless of disordering.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Will check Adams patch, but this should work. As
>>>>> mentioned
>>>>>>>> often
>>>>>>>>> I
>>>>>>>>>> am
>>>>>>>>>>>>>>>>> not convinced on partitioning back for the user
>>>>>>> automatically.
>>>>>>>> I
>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>> this is the real performance eater ;)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 7. Last thought... I'm a little concerned about the
>>>>>>>> performance
>>>>>>>>> of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> range scans when records change in the right table.
>>>>> You've
>>>>>>>> said
>>>>>>>>>> that
>>>>>>>>>>>>>>>>> you've
>>>>>>>>>>>>>>>>>> been using the algorithm you presented in production
>>>>> for a
>>>>>>>>> while.
>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> give us a sense of the performance characteristics
>>>>> you've
>>>>>>>>>> observed?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Make it work, make it fast, make it beautiful. The
>>>>> topmost
>>>>>>>> thing
>>>>>>>>>> here
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> / was correctness. In practice I do not measure the
>>>>>>> performance
>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> range scan. Usual cases I run this with is emitting
>>>>> 500k -
>>>>>>> 1kk
>>>>>>>>> rows
>>>>>>>>>>>>>>>>> on a left hand side change. The range scan is just
>>> the
>>>>> work
>>>>>>> you
>>>>>>>>>> gotta
>>>>>>>>>>>>>>>>> do, also when you pack your data into different
>>> formats,
>>>>>>>> usually
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> rocks performance is very tight to the size of the
>>> data
>>>>> and
>>>>>>> we
>>>>>>>>>> can't
>>>>>>>>>>>>>>>>> really change that. It is more important for users to
>>>>>> prevent
>>>>>>>>>> useless
>>>>>>>>>>>>>>>>> updates to begin with. My left hand side is guarded
>>> to
>>>>> drop
>>>>>>>>> changes
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> are not going to change my join output.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> usually it's:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> drop unused fields and then don't forward if
>>>>>> old.equals(new)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> regarding to the performance of creating an iterator
>>> for
>>>>>>>> smaller
>>>>>>>>>>>>>>>>> fanouts, users can still just do a group by first
>>> then
>>>>>>> anyways.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I could only think of one alternative, but I'm not
>>>>> sure if
>>>>>>>> it's
>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> worse... If the first re-key only needs to preserve
>>> the
>>>>>>>> original
>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>> as I
>>>>>>>>>>>>>>>>>> proposed in #6, then we could store a vector of
>>> keys in
>>>>>> the
>>>>>>>>> value:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Left table:
>>>>>>>>>>>>>>>>>> 1: A,...
>>>>>>>>>>>>>>>>>> 2: B,...
>>>>>>>>>>>>>>>>>> 3: A,...
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Gets re-keyed:
>>>>>>>>>>>>>>>>>> A: [1, 3]
>>>>>>>>>>>>>>>>>> B: [2]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Then, the rhs part of the join would only need a
>>>>> regular
>>>>>>>>>> single-key
>>>>>>>>>>>>>>>>> lookup.
>>>>>>>>>>>>>>>>>> Of course we have to deal with the problem of large
>>>>>> values,
>>>>>>> as
>>>>>>>>>>>>>>> there's
>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>> bound on the number of lhs records that can
>>> reference
>>>>> rhs
>>>>>>>>> records.
>>>>>>>>>>>>>>>>> Offhand,
>>>>>>>>>>>>>>>>>> I'd say we could page the values, so when one row is
>>>>> past
>>>>>>> the
>>>>>>>>>>>>>>>>> threshold, we
>>>>>>>>>>>>>>>>>> append the key for the next page. Then in most
>>> cases,
>>>>> it
>>>>>>> would
>>>>>>>>> be
>>>>>>>>>> a
>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>> key lookup, but for large fan-out updates, it would
>>> be
>>>>> one
>>>>>>> per
>>>>>>>>>> (max
>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>> size)/(avg lhs key size).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This seems more complex, though... Plus, I think
>>>>> there's
>>>>>>> some
>>>>>>>>>> extra
>>>>>>>>>>>>>>>>>> tracking we'd need to do to know when to emit a
>>>>>> retraction.
>>>>>>>> For
>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>> when record 1 is deleted, the re-key table would
>>> just
>>>>> have
>>>>>>> (A:
>>>>>>>>>> [3]).
>>>>>>>>>>>>>>>>> Some
>>>>>>>>>>>>>>>>>> kind of tombstone is needed so that the join result
>>>>> for 1
>>>>>>> can
>>>>>>>>> also
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> retracted.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> That's all!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks so much to both Adam and Jan for the
>>> thoughtful
>>>>>> KIP.
>>>>>>>>> Sorry
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> discussion has been slow.
>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Id say you can just call the vote.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> that happens all the time, and if something comes
>>> up,
>>>>> it
>>>>>>> just
>>>>>>>>>> goes
>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>> to discuss.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> would not expect to much attention with another
>>>>> another
>>>>>>> email
>>>>>>>>> in
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> best Jan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>> Hello Contributors
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I know that 2.1 is about to be released, but I do
>>>>> need
>>>>>> to
>>>>>>>> bump
>>>>>>>>>>>>>>> this to
>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>> visibility up. I am still intending to push this
>>>>> through
>>>>>>>> once
>>>>>>>>>>>>>>>>> contributor
>>>>>>>>>>>>>>>>>>>> feedback is given.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Main points that need addressing:
>>>>>>>>>>>>>>>>>>>> 1) Any way (or benefit) in structuring the current
>>>>>>> singular
>>>>>>>>>> graph
>>>>>>>>>>>>>>> node
>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>> multiple nodes? It has a whopping 25 parameters
>>> right
>>>>>>> now. I
>>>>>>>>> am
>>>>>>>>>> a
>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>> fuzzy
>>>>>>>>>>>>>>>>>>>> on how the optimizations are supposed to work, so
>>> I
>>>>>> would
>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> help on this aspect.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2) Overall strategy for joining + resolving. This
>>>>> thread
>>>>>>> has
>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>> discourse
>>>>>>>>>>>>>>>>>>>> between Jan and I between the current highwater
>>> mark
>>>>>>>> proposal
>>>>>>>>>> and
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>> + reduce proposal. I am of the opinion that we
>>> need
>>>>> to
>>>>>>>>> strictly
>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> chance of out-of-order data and leave none of it
>>> up
>>>>> to
>>>>>> the
>>>>>>>>>>>>>>> consumer.
>>>>>>>>>>>>>>>>> Any
>>>>>>>>>>>>>>>>>>>> comments or suggestions here would also help.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3) Anything else that you see that would prevent
>>> this
>>>>>> from
>>>>>>>>>> moving
>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>>> vote?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> With the Stores.windowStoreBuilder and
>>>>>>>>>>>>>>> Stores.persistentWindowStore,
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> actually only need to specify the amount of
>>> segments
>>>>>> you
>>>>>>>> want
>>>>>>>>>> and
>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>> large
>>>>>>>>>>>>>>>>>>>>> they are. To the best of my understanding, what
>>>>> happens
>>>>>>> is
>>>>>>>>> that
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> segments are automatically rolled over as new
>>> data
>>>>> with
>>>>>>> new
>>>>>>>>>>>>>>>>> timestamps
>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>> created. We use this exact functionality in some
>>> of
>>>>> the
>>>>>>>> work
>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>>>> internally at my company. For reference, this is
>>> the
>>>>>>>> hopping
>>>>>>>>>>>>>>> windowed
>>>>>>>>>>>>>>>>>>> store.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> In the code that I have provided, there are going
>>>>> to be
>>>>>>> two
>>>>>>>>> 24h
>>>>>>>>>>>>>>>>>>> segments.
>>>>>>>>>>>>>>>>>>>>> When a record is put into the windowStore, it
>>> will
>>>>> be
>>>>>>>>> inserted
>>>>>>>>>> at
>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>> T in
>>>>>>>>>>>>>>>>>>>>> both segments. The two segments will always
>>> overlap
>>>>> by
>>>>>>> 12h.
>>>>>>>>> As
>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>> goes on
>>>>>>>>>>>>>>>>>>>>> and new records are added (say at time T+12h+),
>>> the
>>>>>>> oldest
>>>>>>>>>>>> segment
>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> automatically deleted and a new segment created.
>>> The
>>>>>>>> records
>>>>>>>>>> are
>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>> inserted with the context.timestamp(), such that
>>> it
>>>>> is
>>>>>>> the
>>>>>>>>>> record
>>>>>>>>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> the clock time, which is used.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> To the best of my understanding, the timestamps
>>> are
>>>>>>>> retained
>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> restoring from the changelog.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Basically, this is heavy-handed way to deal with
>>> TTL
>>>>>> at a
>>>>>>>>>>>>>>>>> segment-level,
>>>>>>>>>>>>>>>>>>>>> instead of at an individual record level.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Will that work? I expected it to blow up with
>>>>>>>>>> ClassCastException
>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>> similar.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> You either would have to specify the window you
>>>>>>> fetch/put
>>>>>>>> or
>>>>>>>>>>>>>>> iterate
>>>>>>>>>>>>>>>>>>>>>> across all windows the key was found in right?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I just hope the window-store doesn't check
>>>>> stream-time
>>>>>>>> under
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> hoods
>>>>>>>>>>>>>>>>>>>>>> that would be a questionable interface.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> If it does: did you see my comment on checking
>>> all
>>>>> the
>>>>>>>>> windows
>>>>>>>>>>>>>>>>> earlier?
>>>>>>>>>>>>>>>>>>>>>> that would be needed to actually give reasonable
>>>>> time
>>>>>>>>>> gurantees.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Best
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
>>>>> changed
>>>>>>> the
>>>>>>>>>> state
>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> the ProcessorSupplier.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the information. This is indeed
>>>>>> something
>>>>>>>> that
>>>>>>>>>>>>>>> will be
>>>>>>>>>>>>>>>>>>>>>>>>> extremely
>>>>>>>>>>>>>>>>>>>>>>>>> useful for this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for your explanations. That being
>>> said, I
>>>>>> will
>>>>>>>> not
>>>>>>>>>> be
>>>>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>>>>>> ahead
>>>>>>>>>>>>>>>>>>>>>>>>> with an implementation using
>>> reshuffle/groupBy
>>>>>>> solution
>>>>>>>>> as
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> propose.
>>>>>>>>>>>>>>>>>>>>>>>>> That being said, if you wish to implement it
>>>>>> yourself
>>>>>>>> off
>>>>>>>>>> of
>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>> current PR
>>>>>>>>>>>>>>>>>>>>>>>>> and submit it as a competitive alternative, I
>>>>> would
>>>>>>> be
>>>>>>>>> more
>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>> happy to
>>>>>>>>>>>>>>>>>>>>>>>>> help vet that as an alternate solution. As it
>>>>>> stands
>>>>>>>>> right
>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>>> I do
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> really have more time to invest into
>>>>> alternatives
>>>>>>>> without
>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>> strong indication from the binding voters
>>> which
>>>>>> they
>>>>>>>>> would
>>>>>>>>>>>>>>>>> prefer.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hey, total no worries. I think I personally
>>> gave
>>>>> up
>>>>>> on
>>>>>>>> the
>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> some time already, otherwise I would have
>>> pulled
>>>>>> this
>>>>>>>> KIP
>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>>>>> already.
>>>>>>>>>>>>>>>>>>>>>>>> I am currently reimplementing my own DSL
>>> based on
>>>>>>> PAPI.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I will look at finishing up my PR with the
>>>>> windowed
>>>>>>>> state
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>> week or so, exercising it via tests, and
>>> then I
>>>>>> will
>>>>>>>> come
>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>> discussions. In the meantime, I hope that
>>> any of
>>>>>> the
>>>>>>>>>> binding
>>>>>>>>>>>>>>>>> voters
>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
>>>>> updated
>>>>>> it
>>>>>>>>>>>>>>> according
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> latest plan:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>>>>>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I have also updated the KIP PR to use a
>>> windowed
>>>>>>> store.
>>>>>>>>>> This
>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
>>> they
>>>>>> are
>>>>>>>>>>>>>>> completed.
>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
>>>>>> already
>>>>>>>>>> updated
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> PR?
>>>>>>>>>>>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
>>> Missing
>>>>>>>>> something?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
>>> Wang <
>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
>>> is
>>>>> the
>>>>>>>> wrong
>>>>>>>>>>>> link,
>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
>>>>> part of
>>>>>>>>> KIP-258
>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>>>>>> have "handling out-of-order data for source
>>>>>> KTable"
>>>>>>>> such
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>> blindly apply the updates to the
>>> materialized
>>>>>> store,
>>>>>>>>> i.e.
>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>> ordering, we will reject updates that are
>>> older
>>>>>> than
>>>>>>>> the
>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>> key's
>>>>>>>>>>>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
>>> ordering.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
>>>>> Wang <
>>>>>>>>>>>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
>>>>> final
>>>>>>> step
>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
>>>>> with
>>>>>> a
>>>>>>>>> window
>>>>>>>>>>>>>>>>> store),
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>> another current on-going KIP may actually
>>>>> help:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> This is for adding the timestamp into a
>>>>> key-value
>>>>>>>> store
>>>>>>>>>>>>>>> (i.e.
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
>>>>> usage,
>>>>>> as
>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
>>>>>> ,
>>>>>>> is
>>>>>>>>>> that
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>>>> "reject" updates from the source topics if
>>> its
>>>>>>>>> timestamp
>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> smaller
>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>> the current key's latest update timestamp.
>>> I
>>>>>> think
>>>>>>> it
>>>>>>>>> is
>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>> what you have in mind for high watermark
>>> based
>>>>>>>>> filtering,
>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>> need to make sure that the timestamps of
>>> the
>>>>>>> joining
>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> correctly
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> inherited though the whole topology to the
>>>>> final
>>>>>>>> stage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
>>> and
>>>>>> hence
>>>>>>>>>>>>>>>>> non-windowed
>>>>>>>>>>>>>>>>>>>>>> KTables
>>>>>>>>>>>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
>>>>> really
>>>>>>> have
>>>>>>>> a
>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> their joins anyways (
>>>>>>>>>>>>>>>>>>>>>>
>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can just consider non-windowed
>>>>>>> KTable-KTable
>>>>>>>>>>>>>>> non-key
>>>>>>>>>>>>>>>>>>> joins
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
>>> Filipiak
>>>>> <
>>>>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Current highwater mark implementation
>>> would
>>>>>> grow
>>>>>>>>>>>> endlessly
>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary key of original event. It is a
>>> pair
>>>>> of
>>>>>>>> (<this
>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> key>,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
>>> is
>>>>> used
>>>>>>> to
>>>>>>>>>>>>>>>>> differentiate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
>>>>> proposal
>>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> replace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
>>>>> This
>>>>>>> would
>>>>>>>>>> allow
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
>>> time.
>>>>> This
>>>>>>>>> should
>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
>>>>>> should
>>>>>>> be
>>>>>>>>>>>>>>>>> customizable
>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
>>>>> perhaps
>>>>>>> just
>>>>>>>>> 10
>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> window,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
>>> do
>>>>> the
>>>>>>>> trick
>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>> Even
>>>>>>>>>>>>>>>>>>>>>> if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would still like to see the automatic
>>>>>>> repartitioning
>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
>>>>> am a
>>>>>>>> little
>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>> sceptical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> how to determine the window. So esentially
>>> one
>>>>>>> could
>>>>>>>>> run
>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> the rapid change happens near a window
>>>>> border. I
>>>>>>> will
>>>>>>>>>> check
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation in detail, if its
>>>>> problematic, we
>>>>>>>> could
>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>> _all_
>>>>>>>>>>>>>>>>>>>>>>>>>>>> windows on read with not to bad
>>> performance
>>>>>>> impact I
>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if the implementation would be
>>> correct
>>>>> as
>>>>>>> is. I
>>>>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
>>>>>>> timestamp(A)  <
>>>>>>>>>>>>>>>>>>> timestamp(B).
>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> we can't expect that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
>>> -
>>>>>> thanks
>>>>>>>> for
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> diagram, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did really help. You are correct that I
>>> do
>>>>> not
>>>>>>> have
>>>>>>>>> the
>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> key available, and I can see that if it was
>>>>>>> available
>>>>>>>>>> then
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to add and remove events from the
>>> Map.
>>>>>> That
>>>>>>>>> being
>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> encourage
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
>>> for
>>>>>>> clarity
>>>>>>>>> for
>>>>>>>>>>>>>>>>> everyone
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> else.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
>>> really
>>>>> hard
>>>>>>>> work.
>>>>>>>>>> But
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
>>>>>>> original
>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>> join and Group by implemented our own in
>>> PAPI
>>>>>> and
>>>>>>>>>>>> basically
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
>>> missed
>>>>>> that
>>>>>>> in
>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
>>>>> up on
>>>>>>> my
>>>>>>>>> end.
>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>>> finish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
>>>>> week.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> My follow up question for you is, won't
>>> the
>>>>> Map
>>>>>>> stay
>>>>>>>>>>>> inside
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> State
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
>>> changes
>>>>>> have
>>>>>>>>>>>>>>> propagated?
>>>>>>>>>>>>>>>>>>> Isn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
>>>>> state
>>>>>>>> store?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
>>>>> substractor
>>>>>> is
>>>>>>>>> gonna
>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>> `null`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
>>>>> there
>>>>>> is
>>>>>>>>> going
>>>>>>>>>> to
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
>>> this
>>>>>> store
>>>>>>>>>> directly
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
>>> is a
>>>>>>>> regular
>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>> satisfying
>>>>>>>>>>>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
>>>>> join.
>>>>>>> The
>>>>>>>>>>>>>>> Windowed
>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> keeping the values, so for the next
>>> statefull
>>>>>>>> operation
>>>>>>>>>> we
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
>>>>> have
>>>>>> the
>>>>>>>>>> window
>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> the values then.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
>>> custom
>>>>>> group
>>>>>>>> by
>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>> repartitioning to the original primary
>>> key i
>>>>>> think
>>>>>>>> it
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> big time in building efficient apps. Given
>>> the
>>>>>>>> original
>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> understand that we do not have a solid
>>>>> foundation
>>>>>>> to
>>>>>>>>>> build
>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
>>> user.
>>>>>> very
>>>>>>>>>>>>>>>>> unfortunate. I
>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand the decision goes like that. I
>>> do
>>>>> not
>>>>>>>> think
>>>>>>>>>> its
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> decision.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
>>> Prajakta
>>>>>>> Dumbre <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            please remove me from this
>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            On Tue, Sep 11, 2018 at 1:29 PM
>>>>> Jan
>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
>>>>> <mailto:
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Hi Adam,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > give me some time, will make
>>>>> such a
>>>>>>>>> chart.
>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>> time i
>>>>>>>>>>>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            get along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > well with giphy and ruined
>>> all
>>>>> your
>>>>>>>>> charts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Hopefully i can get it done
>>>>> today
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > On 08.09.2018 16:00, Adam
>>>>> Bellemare
>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > Hi Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > I have included a diagram
>>> of
>>>>>> what I
>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > I attempted this back at
>>> the
>>>>>> start
>>>>>>> of
>>>>>>>>> my
>>>>>>>>>> own
>>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > solution, and since I could
>>>>> not
>>>>>> get
>>>>>>>> it
>>>>>>>>> to
>>>>>>>>>>>>>>> work I
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            discarded the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > code. At this point in
>>> time,
>>>>> if
>>>>>> you
>>>>>>>>> wish
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            for your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > groupBy solution, I ask
>>> that
>>>>> you
>>>>>>>> please
>>>>>>>>>>>>>>> create a
>>>>>>>>>>>>>>>>>>>>>> diagram on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > carefully explaining your
>>>>>> solution.
>>>>>>>>>> Please
>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>> free
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the image I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > just posted as a starting
>>>>> point.
>>>>>> I
>>>>>>> am
>>>>>>>>>> having
>>>>>>>>>>>>>>>>> trouble
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            understanding your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > explanations but I think
>>> that
>>>>> a
>>>>>>>>> carefully
>>>>>>>>>>>>>>>>> constructed
>>>>>>>>>>>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            will clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > any misunderstandings.
>>>>>> Alternately,
>>>>>>>>>> please
>>>>>>>>>>>>>>> post a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            comprehensive PR with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > your solution. I can only
>>>>> guess
>>>>>> at
>>>>>>>> what
>>>>>>>>>> you
>>>>>>>>>>>>>>>>> mean, and
>>>>>>>>>>>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            value my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > own
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > time as much as you value
>>>>> yours,
>>>>>> I
>>>>>>>>>> believe
>>>>>>>>>>>> it
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            responsibility to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > provide an implementation
>>>>> instead
>>>>>>> of
>>>>>>>> me
>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > On Sat, Sep 8, 2018 at 8:00
>>>>> AM,
>>>>>> Jan
>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
>>>>> <mailto:
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > > wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Hi James,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> nice to see you beeing
>>>>>> interested.
>>>>>>>>> kafka
>>>>>>>>>>>>>>>>> streams at
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            point supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> all sorts of joins as
>>> long as
>>>>>> both
>>>>>>>>>> streams
>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Adam is currently
>>>>> implementing a
>>>>>>>> join
>>>>>>>>>>>> where a
>>>>>>>>>>>>>>>>> KTable
>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            KTable can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> a one to many relation
>>> ship
>>>>>> (1:n).
>>>>>>>> We
>>>>>>>>>>>> exploit
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> datastore that keeps data
>>>>> sorted
>>>>>> (At
>>>>>>>>> least
>>>>>>>>>>>>>>>>> exposes an
>>>>>>>>>>>>>>>>>>>>>> API to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> stored data in a sorted
>>>>>> fashion).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> I think the technical
>>> caveats
>>>>>> are
>>>>>>>> well
>>>>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > basically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> down to philosophy and API
>>>>>> Design
>>>>>>> (
>>>>>>>>> when
>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>> sees
>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>> newest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            message).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> I have a lengthy track
>>>>> record of
>>>>>>>>> loosing
>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>> kinda
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            arguments within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> streams community and I
>>> have
>>>>> no
>>>>>>> clue
>>>>>>>>>> why.
>>>>>>>>>>>> So
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> literally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            can't wait for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> to churn through this
>>> thread
>>>>> and
>>>>>>>> give
>>>>>>>>>> you
>>>>>>>>>>>>>>>>> opinion on
>>>>>>>>>>>>>>>>>>>>>> how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> the return type of the
>>>>>>> oneToManyJoin
>>>>>>>>> and
>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>> power we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            want to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> the user vs "simplicity"
>>>>> (where
>>>>>>>>>> simplicity
>>>>>>>>>>>>>>> isn't
>>>>>>>>>>>>>>>>>>>>>> really that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            as users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> need to understand it I
>>>>> argue)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> waiting for you to join
>>> in on
>>>>>> the
>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> Best Jan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >> On 07.09.2018 15:49, James
>>>>> Kwan
>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> I am new to this group
>>> and I
>>>>>>> found
>>>>>>>>> this
>>>>>>>>>>>>>>> subject
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            interesting.  Sounds
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> you guys want to
>>> implement a
>>>>>> join
>>>>>>>>>> table of
>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>> streams? Is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > somewhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> I can see the original
>>>>>>> requirement
>>>>>>>> or
>>>>>>>>>>>>>>> proposal?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>> On Sep 7, 2018, at 8:13
>>> AM,
>>>>> Jan
>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <Jan.Filipiak@trivago.com
>>>>> <mailto:
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> On 05.09.2018 22:17,
>>> Adam
>>>>>>>> Bellemare
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> I'm currently testing
>>>>> using a
>>>>>>>>>> Windowed
>>>>>>>>>>>>>>> Store
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> store the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            highwater
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> mark.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> By all indications this
>>>>>> should
>>>>>>>> work
>>>>>>>>>>>> fine,
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> caveat
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            being that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> only resolve
>>> out-of-order
>>>>>>> arrival
>>>>>>>>>> for up
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> size of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the window
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > (ie:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> 24h, 72h, etc). This
>>> would
>>>>>>> remove
>>>>>>>>> the
>>>>>>>>>>>>>>>>> possibility
>>>>>>>>>>>>>>>>>>>>>> of it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > unbounded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> size.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> With regards to Jan's
>>>>>>>> suggestion, I
>>>>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> remain in disagreement.
>>>>>> While I
>>>>>>>> do
>>>>>>>>>> not
>>>>>>>>>>>>>>>>> disagree
>>>>>>>>>>>>>>>>>>>>>> with your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            statement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> there likely to be
>>>>> additional
>>>>>>>> joins
>>>>>>>>>> done
>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            workflow, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> see how you can
>>>>> conclusively
>>>>>>> deal
>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> foreign-key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> changes and subsequent
>>>>>> joins. I
>>>>>>>>> have
>>>>>>>>>>>>>>>>> attempted
>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            think you have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> proposed (without a
>>>>>> high-water,
>>>>>>>>> using
>>>>>>>>>>>>>>>>> groupBy and
>>>>>>>>>>>>>>>>>>>>>> reduce)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            and found
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> that if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> the foreign key changes
>>>>> too
>>>>>>>>> quickly,
>>>>>>>>>> or
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> load
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            stream thread
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> too
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> high, the joined
>>> messages
>>>>>> will
>>>>>>>>> arrive
>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>> and be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            incorrectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> propagated, such that
>>> an
>>>>>>>>> intermediate
>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> represented
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> event.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> Can you shed some light
>>> on
>>>>>> your
>>>>>>>>>> groupBy
>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            There must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> some sort of flaw in it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> I have a suspicion
>>> where it
>>>>>> is,
>>>>>>> I
>>>>>>>>>> would
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            confirm. The idea
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> is bullet proof and it
>>>>> must be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> an implementation mess
>>> up.
>>>>> I
>>>>>>> would
>>>>>>>>>> like
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> clarify
>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we draw a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> conclusion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>    Repartitioning the
>>>>>> scattered
>>>>>>>>> events
>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> partitions is the only
>>> way I
>>>>>> know
>>>>>>>> how
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> conclusively
>>>>>>>>>>>>>>>>>>>>>> deal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> out-of-order events in
>>> a
>>>>>> given
>>>>>>>> time
>>>>>>>>>>>> frame,
>>>>>>>>>>>>>>>>> and to
>>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> eventually consistent
>>> with
>>>>>> the
>>>>>>>>> input
>>>>>>>>>>>>>>> events.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> If you have some code
>>> to
>>>>>> share
>>>>>>>> that
>>>>>>>>>>>>>>>>> illustrates
>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            approach, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> very grateful as it
>>> would
>>>>>>> remove
>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>> misunderstandings
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that I may
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> ah okay you were looking
>>>>> for
>>>>>> my
>>>>>>>>> code.
>>>>>>>>>> I
>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            something easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> readable here as its
>>>>> bloated
>>>>>>> with
>>>>>>>>>>>>>>> OO-patterns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> its anyhow trivial:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> @Override
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      public T apply(K
>>>>> aggKey,
>>>>>> V
>>>>>>>>>> value, T
>>>>>>>>>>>>>>>>>>> aggregate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          Map<U, V>
>>>>>>>>> currentStateAsMap =
>>>>>>>>>>>>>>>>>>>>>> asMap(aggregate);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <<
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            imaginary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          U toModifyKey =
>>>>>>>>>>>>>>> mapper.apply(value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << this is
>>> the
>>>>>>> place
>>>>>>>>>> where
>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            gonna have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> and why you probably
>>>>> couldn't
>>>>>> do
>>>>>>>> it.
>>>>>>>>>> we
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>> to find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            a solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> I didn't realize that
>>> yet.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << we
>>>>> propagate
>>>>>> the
>>>>>>>>>> field in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> joiner, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            that we can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > pick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> it up in an aggregate.
>>>>>> Probably
>>>>>>>> you
>>>>>>>>>> have
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> thought
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            this in your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> approach right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << I am
>>> very
>>>>> open
>>>>>>> to
>>>>>>>>>> find a
>>>>>>>>>>>>>>>>> generic
>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            here. In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> honest opinion this is
>>>>> broken
>>>>>> in
>>>>>>>>>>>>>>>>>>> KTableImpl.GroupBy
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            looses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> and only maintains the
>>>>>> aggregate
>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              << I
>>>>> abstracted
>>>>>> it
>>>>>>>> away
>>>>>>>>>> back
>>>>>>>>>>>>>>>>> then way
>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> of oneToMany join. That
>>> is
>>>>>> why I
>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>> realize
>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            significance here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              <<
>>> Opinions?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          for (V m :
>>>>> current)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>> currentStateAsMap.put(mapper.apply(m),
>>>>>>>>>>>> m);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          if (isAdder)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>> currentStateAsMap.put(toModifyKey,
>>>>>>>>>>>> value);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          else
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>                  return
>>>>> null;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>          retrun
>>>>>>>>>>>>>>>>> asAggregateType(currentStateAsMap)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>      }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> On Wed, Sep 5, 2018 at
>>>>> 3:35
>>>>>> PM,
>>>>>>>> Jan
>>>>>>>>>>>>>>> Filipiak
>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Jan.Filipiak@trivago.com
>>>>> <mailto:
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>> Thanks Adam for
>>> bringing
>>>>>>> Matthias
>>>>>>>>> to
>>>>>>>>>>>>>>> speed!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> about the
>>> differences. I
>>>>>> think
>>>>>>>>>>>> re-keying
>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            optional at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> best.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I would say we return
>>> a
>>>>>>>>>> KScatteredTable
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> reshuffle()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>> KTable<originalKey,Joined>
>>>>>> to
>>>>>>>> make
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            repartitioning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I am also in a big
>>>>> favour of
>>>>>>>> doing
>>>>>>>>>> the
>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> group
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> by instead high water
>>>>> mark
>>>>>>>>> tracking.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> Just because unbounded
>>>>>> growth
>>>>>>> is
>>>>>>>>>> just
>>>>>>>>>>>>>>> scary
>>>>>>>>>>>>>>>>> + It
>>>>>>>>>>>>>>>>>>>>>> saves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> stuff.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> I think the
>>> abstraction
>>>>> of
>>>>>>>> always
>>>>>>>>>>>>>>>>> repartitioning
>>>>>>>>>>>>>>>>>>>>>> back is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            just not so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> strong. Like the work
>>> has
>>>>>> been
>>>>>>>>> done
>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            back and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> grouping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> by something else
>>>>> afterwards
>>>>>>> is
>>>>>>>>>> really
>>>>>>>>>>>>>>>>> common.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> On 05.09.2018 13:49,
>>> Adam
>>>>>>>>> Bellemare
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>> Hi Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Thank you for your
>>>>>> feedback,
>>>>>>> I
>>>>>>>> do
>>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> While name spacing
>>>>> would be
>>>>>>>>>> possible,
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> user headers what
>>>>> implies
>>>>>> a
>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> namespace for now to
>>>>> avoid
>>>>>>> the
>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the future, we can
>>>>> still
>>>>>> add
>>>>>>>>> name
>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Agreed. I will go
>>> with
>>>>>>> using a
>>>>>>>>>>>> reserved
>>>>>>>>>>>>>>>>> string
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            document it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> My main concern about
>>>>> the
>>>>>>>> design
>>>>>>>>> it
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> type of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> understood the
>>> proposal
>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> In your example, you
>>>>> have
>>>>>>>> table1
>>>>>>>>>> and
>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>> swapped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            Here is how it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> works
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> currently:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 1) table1 has the
>>>>> records
>>>>>>> that
>>>>>>>>>> contain
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> foreign key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            within their
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> value.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 input stream:
>>>>>>>>>> <a,(fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table2 input stream:
>>>>> <A,X>,
>>>>>>>> <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 2) A Value mapper is
>>>>>> required
>>>>>>>> to
>>>>>>>>>>>> extract
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 foreign key
>>>>> mapper:
>>>>>> (
>>>>>>>>> value
>>>>>>>>>> =>
>>>>>>>>>>>>>>>>> value.fk
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <http://value.fk> )
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> The mapper is
>>> applied to
>>>>>> each
>>>>>>>>>> element
>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            new combined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> key is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> made:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> table1 mapped: <A-a,
>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            <B-c,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 3) The rekeyed events
>>>>> are
>>>>>>>>>>>> copartitioned
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> table2:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> a) Stream Thread with
>>>>>>> Partition
>>>>>>>>> 0:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> RepartitionedTable1:
>>>>> <A-a,
>>>>>>>>>>>>>>> (fk=A,bar=1)>,
>>>>>>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            (fk=A,bar=2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> b) Stream Thread with
>>>>>>> Partition
>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> RepartitionedTable1:
>>>>> <B-c,
>>>>>>>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> 4) From here, they
>>> can
>>>>> be
>>>>>>>> joined
>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            applying the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> joiner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> function.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> At this point, Jan's
>>>>> design
>>>>>>> and
>>>>>>>>> my
>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>> deviate. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            design goes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> repartition the data
>>>>>>> post-join
>>>>>>>>> and
>>>>>>>>>>>>>>> resolve
>>>>>>>>>>>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            arrival of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> records,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> finally returning the
>>>>> data
>>>>>>>> keyed
>>>>>>>>>> just
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> original key.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            I do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> expose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> CombinedKey or any of
>>>>> the
>>>>>>>>> internals
>>>>>>>>>>>>>>>>> outside of
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            joinOnForeignKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> function. This does
>>> make
>>>>>> for
>>>>>>>>> larger
>>>>>>>>>>>>>>>>> footprint,
>>>>>>>>>>>>>>>>>>>>>> but it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            removes all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> agency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> for resolving
>>>>> out-of-order
>>>>>>>>> arrivals
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            CombinedKeys from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> user. I believe that
>>>>> this
>>>>>>> makes
>>>>>>>>> the
>>>>>>>>>>>>>>>>> function
>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to use.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Let me know if this
>>>>> helps
>>>>>>>> resolve
>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>> questions,
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            please feel
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> add anything else on
>>>>> your
>>>>>>> mind.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> On Tue, Sep 4, 2018
>>> at
>>>>> 8:36
>>>>>>> PM,
>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>> matthias@confluent.io
>>>>>>> <mailto:
>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> I am just catching
>>> up
>>>>> on
>>>>>>> this
>>>>>>>>>>>> thread. I
>>>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            everything so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> far,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> but want to share
>>>>> couple
>>>>>> of
>>>>>>>>>> initial
>>>>>>>>>>>>>>>>> thoughts:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Headers: I think
>>> there
>>>>> is
>>>>>> a
>>>>>>>>>>>> fundamental
>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            between header
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> usage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> in this KIP and
>>> KP-258.
>>>>>> For
>>>>>>>> 258,
>>>>>>>>>> we
>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>> headers
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> are owned by Kafka
>>>>> Streams
>>>>>>> and
>>>>>>>>>> nobody
>>>>>>>>>>>>>>>>> else is
>>>>>>>>>>>>>>>>>>>>>> supposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> them. In fact, no
>>> user
>>>>>>> header
>>>>>>>>> are
>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            changelog topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> thus, there are not
>>>>>>> conflicts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Nevertheless, I
>>> don't
>>>>> see
>>>>>> a
>>>>>>>> big
>>>>>>>>>> issue
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headers within
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> As long as we
>>> document
>>>>> it,
>>>>>>> we
>>>>>>>>> can
>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>> "reserved"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            header keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> users are not
>>> allowed
>>>>> to
>>>>>> use
>>>>>>>>> when
>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>> data with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > Streams.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> IMHO, this should be
>>>>> ok.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> I think there is a
>>> safe
>>>>>> way
>>>>>>> to
>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>> conflicts,
>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > headers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> only needed in
>>>>> internal
>>>>>>>> topics
>>>>>>>>> (I
>>>>>>>>>>>>>>> think):
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> For internal and
>>>>>> changelog
>>>>>>>>>> topics,
>>>>>>>>>>>> we
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>> namespace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            all headers:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> * user-defined
>>> headers
>>>>>> are
>>>>>>>>>>>> namespaced
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>> "external."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> * internal headers
>>> are
>>>>>>>>>> namespaced as
>>>>>>>>>>>>>>>>>>>>>> "internal." +
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            headerKey
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> While name spacing
>>>>> would
>>>>>> be
>>>>>>>>>>>> possible,
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> deserialize
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> user headers what
>>>>> implies
>>>>>> a
>>>>>>>>>> runtime
>>>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            suggest to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> namespace for now to
>>>>> avoid
>>>>>>> the
>>>>>>>>>>>>>>> overhead.
>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > problem in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the future, we can
>>>>> still
>>>>>> add
>>>>>>>>> name
>>>>>>>>>>>>>>> spacing
>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> My main concern
>>> about
>>>>> the
>>>>>>>> design
>>>>>>>>>> it
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            result KTable:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> If I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> understood the
>>> proposal
>>>>>>>>> correctly,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K1,V1>
>>> table1 =
>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K2,V2>
>>> table2 =
>>>>> ...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> KTable<K1,V3>
>>>>> joinedTable
>>>>>> =
>>>>>>>>>>>>>>>>>>>>>> table1.join(table2,...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> implies that the
>>>>>>> `joinedTable`
>>>>>>>>> has
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            left input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> IMHO, this does not
>>>>> work
>>>>>>>> because
>>>>>>>>>> if
>>>>>>>>>>>>>>> table2
>>>>>>>>>>>>>>>>>>>>>> contains
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            multiple rows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> join with a record
>>> in
>>>>>> table1
>>>>>>>>>> (what is
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> main
>>>>>>>>>>>>>>>>>>>>>> purpose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > foreign
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> join), the result
>>> table
>>>>>>> would
>>>>>>>>> only
>>>>>>>>>>>>>>>>> contain a
>>>>>>>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            join result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> multiple.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Example:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table1 input stream:
>>>>> <A,X>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> table2 input stream:
>>>>>>>> <a,(A,1)>,
>>>>>>>>>>>>>>> <b,(A,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> We use table2 value
>>> a
>>>>>>> foreign
>>>>>>>>> key
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> table1
>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            "A" joins).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> result key is the
>>> same
>>>>> key
>>>>>>> as
>>>>>>>>> key
>>>>>>>>>> of
>>>>>>>>>>>>>>>>> table1,
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            implies that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> result can either be
>>>>> <A,
>>>>>>>>>> join(X,1)>
>>>>>>>>>>>> or
>>>>>>>>>>>>>>> <A,
>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > both.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Because the share
>>> the
>>>>> same
>>>>>>>> key,
>>>>>>>>>>>>>>> whatever
>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            we emit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> overwrite the
>>> previous
>>>>>>> result.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> This is the reason
>>> why
>>>>> Jan
>>>>>>>>>> originally
>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > combination
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> both primary keys of
>>>>> the
>>>>>>> input
>>>>>>>>>> tables
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            output table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> makes the keys of
>>> the
>>>>>> output
>>>>>>>>> table
>>>>>>>>>>>>>>> unique
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            store both in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> output table:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Result would be
>>> <A-a,
>>>>>>>>> join(X,1)>,
>>>>>>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> On 9/4/18 1:36 PM,
>>> Jan
>>>>>>>> Filipiak
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>> Just on remark here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> The high-watermark
>>>>> could
>>>>>> be
>>>>>>>>>>>>>>> disregarded.
>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> depends on the
>>> size of
>>>>>> the
>>>>>>>>>>>> aggregated
>>>>>>>>>>>>>>>>> map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> Only 1 element long
>>>>> maps
>>>>>>>> would
>>>>>>>>> be
>>>>>>>>>>>>>>>>> unpacked
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            forwarded. 0
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > element
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> maps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> would be published
>>> as
>>>>>>> delete.
>>>>>>>>> Any
>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>> count
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> of map entries is
>>> in
>>>>>>> "waiting
>>>>>>>>> for
>>>>>>>>>>>>>>> correct
>>>>>>>>>>>>>>>>>>>>>> deletes to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > arrive"-state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> On 04.09.2018
>>> 21:29,
>>>>> Adam
>>>>>>>>>> Bellemare
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> It does look like I
>>>>> could
>>>>>>>>> replace
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            repartition store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> highwater store
>>> with
>>>>> a
>>>>>>>> groupBy
>>>>>>>>>> and
>>>>>>>>>>>>>>>>> reduce.
>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            it looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> still need to
>>> store
>>>>> the
>>>>>>>>>> highwater
>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>> within
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            materialized
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> compare the
>>> arrival of
>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>> (assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> THIS is
>>> correct...).
>>>>> This
>>>>>>> in
>>>>>>>>>> effect
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            design I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>> just with the two
>>>>> tables
>>>>>>>> merged
>>>>>>>>>>>>>>> together.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> -- Guozhang
>>>>>>
>>>>>
>>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi All

Sorry for the delay - holidays and all. I have since updated the KIP with
John's original suggestion and have pruned a number of the no longer
relevant diagrams. Any more comments would be welcomed, otherwise I will
look to kick off the vote again shortly.

Thanks
Adam

On Mon, Dec 17, 2018 at 7:06 PM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi John and Guozhang
>
> Ah yes, I lost that in the mix! Thanks for the convergent solutions - I do
> think that the attachment that John included makes for a better design. It
> should also help with overall performance as very high-cardinality foreign
> keyed data (say millions of events with the same entity) will be able to
> leverage the multiple nodes for join functionality instead of having it all
> performed in one node. There is still a bottleneck in the right table
> having to propagate all those events, but with slimmer structures, less IO
> and no need to perform the join I think the throughput will be much higher
> in those scenarios.
>
> Okay, I am convinced. I will update the KIP accordingly to a Gliffy
> version of John's diagram and ensure that the example flow matches
> correctly. Then I can go back to working on the PR to match the diagram.
>
> Thanks both of you for all the help - very much appreciated.
>
> Adam
>
>
>
>
>
>
>
> On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com> wrote:
>
>> Hi John,
>>
>> Just made a pass on your diagram (nice hand-drawing btw!), and obviously
>> we
>> are thinking about the same thing :) A neat difference that I like, is
>> that
>> in the pre-join repartition topic we can still send message in the format
>> of `K=k, V=(i=2)` while using "i" as the partition key in
>> StreamsPartition,
>> this way we do not need to even augment the key for the repartition topic,
>> but just do a projection on the foreign key part but trim all other
>> fields:
>> as long as we still materialize the store as `A-2` co-located with the
>> right KTable, that is fine.
>>
>> As I mentioned in my previous email, I also think this has a few
>> advantages
>> on saving over-the-wire bytes as well as disk bytes.
>>
>> Guozhang
>>
>>
>> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io> wrote:
>>
>> > Hi Guozhang,
>> >
>> > Thanks for taking a look! I think Adam's already addressed your
>> questions
>> > as well as I could have.
>> >
>> > Hi Adam,
>> >
>> > Thanks for updating the KIP. It looks great, especially how all the
>> > need-to-know information is right at the top, followed by the details.
>> >
>> > Also, thanks for that high-level diagram. Actually, now that I'm looking
>> > at it, I think part of my proposal got lost in translation, although I
>> do
>> > think that what you have there is also correct.
>> >
>> > I sketched up a crude diagram based on yours and attached it to the KIP
>> > (I'm not sure if attached or inline images work on the mailing list):
>> >
>> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
>> > . It's also attached to this email for convenience.
>> >
>> > Hopefully, you can see how it's intended to line up, and which parts are
>> > modified.
>> > At a high level, instead of performing the join on the right-hand side,
>> > we're essentially just registering interest, like "LHS key A wishes to
>> > receive updates for RHS key 2". Then, when there is a new "interest" or
>> any
>> > updates to the RHS records, it "broadcasts" its state back to the LHS
>> > records who are interested in it.
>> >
>> > Thus, instead of sending the LHS values to the RHS joiner workers and
>> then
>> > sending the join results back to the LHS worke be co-partitioned and
>> > validated, we instead only send the LHS *keys* to the RHS workers and
>> then
>> > only the RHS k/v back to be joined by the LHS worker.
>> >
>> > I've been considering both your diagram and mine, and I *think* what I'm
>> > proposing has a few advantages.
>> >
>> > Here are some points of interest as you look at the diagram:
>> > * When we extract the foreign key and send it to the Pre-Join
>> Repartition
>> > Topic, we can send only the FK/PK pair. There's no need to worry about
>> > custom partitioner logic, since we can just use the foreign key plainly
>> as
>> > the repartition record key. Also, we save on transmitting the LHS value,
>> > since we only send its key in this step.
>> > * We also only need to store the RHSKey:LHSKey mapping in the
>> > MaterializedSubscriptionStore, saving on disk. We can use the same rocks
>> > key format you proposed and the same algorithm involving range scans
>> when
>> > the RHS records get updated.
>> > * Instead of joining on the right side, all we do is compose a
>> > re-repartition record so we can broadcast the RHS k/v pair back to the
>> > original LHS partition. (this is what the "rekey" node is doing)
>> > * Then, there is a special kind of Joiner that's co-resident in the same
>> > StreamTask as the LHS table, subscribed to the Post-Join Repartition
>> Topic.
>> > ** This Joiner is *not* triggered directly by any changes in the LHS
>> > KTable. Instead, LHS events indirectly trigger the join via the whole
>> > lifecycle.
>> > ** For each event arriving from the Post-Join Repartition Topic, the
>> > Joiner looks up the corresponding record in the LHS KTable. It validates
>> > the FK as you noted, discarding any inconsistent events. Otherwise, it
>> > unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the join
>> > result
>> > ** Note that the Joiner itself is stateless, so materializing the join
>> > result is optional, just as with the 1:1 joins.
>> >
>> > So in summary:
>> > * instead of transmitting the LHS keys and values to the right and the
>> > JoinResult back to the left, we only transmit the LHS keys to the right
>> and
>> > the RHS values to the left. Assuming the average RHS value is on smaller
>> > than or equal to the average join result size, it's a clear win on
>> broker
>> > traffic. I think this is actually a reasonable assumption, which we can
>> > discuss more if you're suspicious.
>> > * we only need one copy of the data (the left and right tables need to
>> be
>> > materialized) and one extra copy of the PK:FK pairs in the Materialized
>> > Subscription Store. Materializing the join result is optional, just as
>> with
>> > the existing 1:1 joins.
>> > * we still need the fancy range-scan algorithm on the right to locate
>> all
>> > interested LHS keys when a RHS value is updated, but we don't need a
>> custom
>> > partitioner for either repartition topic (this is of course a
>> modification
>> > we could make to your version as well)
>> >
>> > How does this sound to you? (And did I miss anything?)
>> > -John
>> >
>> > On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>> > wrote:
>> >
>> >> Hi John & Guozhang
>> >>
>> >> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the
>> KIP,
>> >> pruned much of what I wrote and put a simplified diagram near the top
>> to
>> >> illustrate the workflow. I encapsulated Jan's content at the bottom of
>> the
>> >> document. I believe it is simpler to read by far now.
>> >>
>> >> @Guozhang Wang <wa...@gmail.com>:
>> >> > #1: rekey left table
>> >> >   -> source from the left upstream, send to rekey-processor to
>> generate
>> >> combined key, and then sink to copartition topic.
>> >> Correct.
>> >>
>> >> > #2: first-join with right table
>> >> >   -> source from the right table upstream, materialize the right
>> table.
>> >> >   -> source from the co-partition topic, materialize the rekeyed left
>> >> table, join with the right table, rekey back, and then sink to the
>> >> rekeyed-back topic.
>> >> Almost - I cleared up the KIP. We do not rekey back yet, as I need the
>> >> Foreign-Key value generated in #1 above to compare in the resolution
>> >> stage.
>> >>
>> >> > #3: second join
>> >> >    -> source from the rekeyed-back topic, materialize the rekeyed
>> back
>> >> table.
>> >> >   -> source from the left upstream, materialize the left table, join
>> >> with
>> >> the rekeyed back table.
>> >> Almost - As each event comes in, we just run it through a stateful
>> >> processor that checks the original ("This") KTable for the key. The
>> value
>> >> payload then has the foreignKeyExtractor applied again as in Part #1
>> >> above,
>> >> and gets the current foreign key. Then we compare it to the joined
>> event
>> >> that we are currently resolving. If they have the same foreign-key,
>> >> propagate the result out. If they don't, throw the event away.
>> >>
>> >> The end result is that we do need to materialize 2 additional tables
>> >> (left/this-combinedkey table, and the final Joined table) as I've
>> >> illustrated in the updated KIP. I hope the diagram clears it up a lot
>> >> better. Please let me know.
>> >>
>> >> Thanks again
>> >> Adam
>> >>
>> >>
>> >>
>> >>
>> >> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
>> wrote:
>> >>
>> >> > John,
>> >> >
>> >> > Thanks a lot for the suggestions on refactoring the wiki, I agree
>> with
>> >> you
>> >> > that we should consider the KIP proposal to be easily understood by
>> >> anyone
>> >> > in the future to read, and hence should provide a good summary on the
>> >> > user-facing interfaces, as well as rejected alternatives to represent
>> >> > briefly "how we came a long way to this conclusion, and what we have
>> >> > argued, disagreed, and agreed about, etc" so that readers do not
>> need to
>> >> > dig into the DISCUSS thread to get all the details. We can, of
>> course,
>> >> keep
>> >> > the implementation details like "workflows" on the wiki page as a
>> >> addendum
>> >> > section since it also has correlations.
>> >> >
>> >> > Regarding your proposal on comment 6): that's a very interesting
>> idea!
>> >> Just
>> >> > to clarify that I understands it fully correctly: the proposal's
>> >> resulted
>> >> > topology is still the same as the current proposal, where we will
>> have 3
>> >> > sub-topologies for this operator:
>> >> >
>> >> > #1: rekey left table
>> >> >    -> source from the left upstream, send to rekey-processor to
>> generate
>> >> > combined key, and then sink to copartition topic.
>> >> >
>> >> > #2: first-join with right table
>> >> >    -> source from the right table upstream, materialize the right
>> table.
>> >> >    -> source from the co-partition topic, materialize the rekeyed
>> left
>> >> > table, join with the right table, rekey back, and then sink to the
>> >> > rekeyed-back topic.
>> >> >
>> >> > #3: second join
>> >> >    -> source from the rekeyed-back topic, materialize the rekeyed
>> back
>> >> > table.
>> >> >    -> source from the left upstream, materialize the left table, join
>> >> with
>> >> > the rekeyed back table.
>> >> >
>> >> > Sub-topology #1 and #3 may be merged to a single sub-topology since
>> >> both of
>> >> > them read from the left table source stream. In this workflow, we
>> need
>> >> to
>> >> > materialize 4 tables (left table in #3, right table in #2, rekeyed
>> left
>> >> > table in #2, rekeyed-back table in #3), and 2 repartition topics
>> >> > (copartition topic, rekeyed-back topic).
>> >> >
>> >> > Compared with Adam's current proposal in the workflow overview, it
>> has
>> >> the
>> >> > same num.materialize tables (left table, rekeyed left table, right
>> >> table,
>> >> > out-of-ordering resolver table), and same num.internal topics (two).
>> The
>> >> > advantage is that on the copartition topic, we can save bandwidth by
>> not
>> >> > sending value, and in #2 the rekeyed left table is smaller since we
>> do
>> >> not
>> >> > have any values to materialize. Is that right?
>> >> >
>> >> >
>> >> > Guozhang
>> >> >
>> >> >
>> >> >
>> >> > On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
>> wrote:
>> >> >
>> >> > > Hi Adam,
>> >> > >
>> >> > > Given that the committers are all pretty busy right now, I think
>> that
>> >> it
>> >> > > would help if you were to refactor the KIP a little to reduce the
>> >> > workload
>> >> > > for reviewers.
>> >> > >
>> >> > > I'd recommend the following changes:
>> >> > > * relocate all internal details to a section at the end called
>> >> something
>> >> > > like "Implementation Notes" or something like that.
>> >> > > * rewrite the rest of the KIP to be a succinct as possible and
>> mention
>> >> > only
>> >> > > publicly-facing API changes.
>> >> > > ** for example, the interface that you've already listed there, as
>> >> well
>> >> > as
>> >> > > a textual description of the guarantees we'll be providing (join
>> >> result
>> >> > is
>> >> > > copartitioned with the LHS, and the join result is guaranteed
>> correct)
>> >> > >
>> >> > > A good target would be that the whole main body of the KIP,
>> including
>> >> > > Status, Motivation, Proposal, Justification, and Rejected
>> Alternatives
>> >> > all
>> >> > > fit "above the fold" (i.e., all fit on the screen at a comfortable
>> >> zoom
>> >> > > level).
>> >> > > I think the only real Rejected Alternative that bears mention at
>> this
>> >> > point
>> >> > > is KScatteredTable, which you could just include the executive
>> >> summary on
>> >> > > (no implementation details), and link to extra details in the
>> >> > > Implementation Notes section.
>> >> > >
>> >> > > Taking a look at the wiki page, ~90% of the text there is internal
>> >> > detail,
>> >> > > which is useful for the dubious, but doesn't need to be ratified
>> in a
>> >> > vote
>> >> > > (and would be subject to change without notice in the future
>> anyway).
>> >> > > There's also a lot of conflicting discussion, as you've very
>> >> respectfully
>> >> > > tried to preserve the original proposal from Jan while adding your
>> >> own.
>> >> > > Isolating all this information in a dedicated section at the bottom
>> >> frees
>> >> > > the voters up to focus on the public API part of the proposal,
>> which
>> >> is
>> >> > > really all they need to consider.
>> >> > >
>> >> > > Plus, it'll be clear to future readers which parts of the document
>> are
>> >> > > enduring, and which parts are a snapshot of our implementation
>> >> thinking
>> >> > at
>> >> > > the time.
>> >> > >
>> >> > > I'm suggesting this because I suspect that the others haven't made
>> >> time
>> >> > to
>> >> > > review it partly because it seems daunting. If it seems like it
>> would
>> >> be
>> >> > a
>> >> > > huge time investment to review, people will just keep putting it
>> off.
>> >> But
>> >> > > if the KIP is a single page, then they'll be more inclined to give
>> it
>> >> a
>> >> > > read.
>> >> > >
>> >> > > Honestly, I don't think the KIP itself is that controversial (apart
>> >> from
>> >> > > the scattered table thing (sorry, Jan) ). Most of the discussion
>> has
>> >> been
>> >> > > around the implementation, which we can continue more effectively
>> in
>> >> a PR
>> >> > > once the KIP has passed.
>> >> > >
>> >> > > How does that sound?
>> >> > > -John
>> >> > >
>> >> > > On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
>> >> adam.bellemare@gmail.com
>> >> > >
>> >> > > wrote:
>> >> > >
>> >> > > > 1) I believe that the resolution mechanism John has proposed is
>> >> > > sufficient
>> >> > > > - it is clean and easy and doesn't require additional RocksDB
>> >> stores,
>> >> > > which
>> >> > > > reduces the footprint greatly. I don't think we need to resolve
>> >> based
>> >> > on
>> >> > > > timestamp or offset anymore, but if we decide to do to that
>> would be
>> >> > > within
>> >> > > > the bounds of the existing API.
>> >> > > >
>> >> > > > 2) Is the current API sufficient, or does it need to be altered
>> to
>> >> go
>> >> > > back
>> >> > > > to vote?
>> >> > > >
>> >> > > > 3) KScatteredTable implementation can always be added in a future
>> >> > > revision.
>> >> > > > This API does not rule it out. This implementation of this
>> function
>> >> > would
>> >> > > > simply be replaced with `KScatteredTable.resolve()` while still
>> >> > > maintaining
>> >> > > > the existing API, thereby giving both features as Jan outlined
>> >> earlier.
>> >> > > > Would this work?
>> >> > > >
>> >> > > >
>> >> > > > Thanks Guozhang, John and Jan
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <john@confluent.io
>> >
>> >> > wrote:
>> >> > > >
>> >> > > > > Hi, all,
>> >> > > > >
>> >> > > > > >> In fact, we
>> >> > > > > >> can just keep a single final-result store with timestamps
>> and
>> >> > reject
>> >> > > > > values
>> >> > > > > >> that have a smaller timestamp, is that right?
>> >> > > > >
>> >> > > > > > Which is the correct output should at least be decided on the
>> >> > offset
>> >> > > of
>> >> > > > > > the original message.
>> >> > > > >
>> >> > > > > Thanks for this point, Jan.
>> >> > > > >
>> >> > > > > KIP-258 is merely to allow embedding the record timestamp  in
>> the
>> >> k/v
>> >> > > > > store,
>> >> > > > > as well as providing a storage-format upgrade path.
>> >> > > > >
>> >> > > > > I might have missed it, but I think we have yet to discuss
>> whether
>> >> > it's
>> >> > > > > safe
>> >> > > > > or desirable just to swap topic-ordering our for
>> >> timestamp-ordering.
>> >> > > This
>> >> > > > > is
>> >> > > > > a very deep topic, and I think it would only pollute the
>> current
>> >> > > > > discussion.
>> >> > > > >
>> >> > > > > What Adam has proposed is safe, given the *current* ordering
>> >> > semantics
>> >> > > > > of the system. If we can agree on his proposal, I think we can
>> >> merge
>> >> > > the
>> >> > > > > feature well before the conversation about timestamp ordering
>> even
>> >> > > takes
>> >> > > > > place, much less reaches a conclusion. In the mean time, it
>> would
>> >> > seem
>> >> > > to
>> >> > > > > be unfortunate to have one join operator with different
>> ordering
>> >> > > > semantics
>> >> > > > > from every other KTable operator.
>> >> > > > >
>> >> > > > > If and when that timestamp discussion takes place, many (all?)
>> >> KTable
>> >> > > > > operations
>> >> > > > > will need to be updated, rendering the many:one join a small
>> >> marginal
>> >> > > > cost.
>> >> > > > >
>> >> > > > > And, just to plug it again, I proposed an algorithm above that
>> I
>> >> > > believe
>> >> > > > > provides
>> >> > > > > correct ordering without any additional metadata, and
>> regardless
>> >> of
>> >> > the
>> >> > > > > ordering semantics. I didn't bring it up further, because I
>> felt
>> >> the
>> >> > > KIP
>> >> > > > > only needs
>> >> > > > > to agree on the public API, and we can discuss the
>> implementation
>> >> at
>> >> > > > > leisure in
>> >> > > > > a PR...
>> >> > > > >
>> >> > > > > Thanks,
>> >> > > > > -John
>> >> > > > >
>> >> > > > >
>> >> > > > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
>> >> > Jan.Filipiak@trivago.com
>> >> > > >
>> >> > > > > wrote:
>> >> > > > >
>> >> > > > > >
>> >> > > > > >
>> >> > > > > > On 10.12.2018 07:42, Guozhang Wang wrote:
>> >> > > > > > > Hello Adam / Jan / John,
>> >> > > > > > >
>> >> > > > > > > Sorry for being late on this thread! I've finally got some
>> >> time
>> >> > > this
>> >> > > > > > > weekend to cleanup a load of tasks on my queue (actually
>> I've
>> >> > also
>> >> > > > > > realized
>> >> > > > > > > there are a bunch of other things I need to enqueue while
>> >> > cleaning
>> >> > > > them
>> >> > > > > > up
>> >> > > > > > > --- sth I need to improve on my side). So here are my
>> >> thoughts:
>> >> > > > > > >
>> >> > > > > > > Regarding the APIs: I like the current written API in the
>> KIP.
>> >> > More
>> >> > > > > > > generally I'd prefer to keep the 1) one-to-many join
>> >> > > functionalities
>> >> > > > as
>> >> > > > > > > well as 2) other join types than inner as separate KIPs
>> since
>> >> 1)
>> >> > > may
>> >> > > > > > worth
>> >> > > > > > > a general API refactoring that can benefit not only
>> foreignkey
>> >> > > joins
>> >> > > > > but
>> >> > > > > > > collocate joins as well (e.g. an extended proposal of
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
>> >> > > > > > ),
>> >> > > > > > > and I'm not sure if other join types would actually be
>> needed
>> >> > > (maybe
>> >> > > > > left
>> >> > > > > > > join still makes sense), so it's better to
>> >> > > > > wait-for-people-to-ask-and-add
>> >> > > > > > > than add-sth-that-no-one-uses.
>> >> > > > > > >
>> >> > > > > > > Regarding whether we enforce step 3) / 4) v.s. introducing
>> a
>> >> > > > > > > KScatteredTable for users to inject their own optimization:
>> >> I'd
>> >> > > > prefer
>> >> > > > > to
>> >> > > > > > > do the current option as-is, and my main rationale is for
>> >> > > > optimization
>> >> > > > > > > rooms inside the Streams internals and the API
>> succinctness.
>> >> For
>> >> > > > > advanced
>> >> > > > > > > users who may indeed prefer KScatteredTable and do their
>> own
>> >> > > > > > optimization,
>> >> > > > > > > while it is too much of the work to use Processor API
>> >> directly, I
>> >> > > > think
>> >> > > > > > we
>> >> > > > > > > can still extend the current API to support it in the
>> future
>> >> if
>> >> > it
>> >> > > > > > becomes
>> >> > > > > > > necessary.
>> >> > > > > >
>> >> > > > > > no internal optimization potential. it's a myth
>> >> > > > > >
>> >> > > > > > ¯\_(ツ)_/¯
>> >> > > > > >
>> >> > > > > > :-)
>> >> > > > > >
>> >> > > > > > >
>> >> > > > > > > Another note about step 4) resolving out-of-ordering data,
>> as
>> >> I
>> >> > > > > mentioned
>> >> > > > > > > before I think with KIP-258 (embedded timestamp with
>> key-value
>> >> > > store)
>> >> > > > > we
>> >> > > > > > > can actually make this step simpler than the current
>> >> proposal. In
>> >> > > > fact,
>> >> > > > > > we
>> >> > > > > > > can just keep a single final-result store with timestamps
>> and
>> >> > > reject
>> >> > > > > > values
>> >> > > > > > > that have a smaller timestamp, is that right?
>> >> > > > > >
>> >> > > > > > Which is the correct output should at least be decided on the
>> >> > offset
>> >> > > of
>> >> > > > > > the original message.
>> >> > > > > >
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > > That's all I have in mind now. Again, great appreciation to
>> >> Adam
>> >> > to
>> >> > > > > make
>> >> > > > > > > such HUGE progress on this KIP!
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > > Guozhang
>> >> > > > > > >
>> >> > > > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
>> >> > > > Jan.Filipiak@trivago.com>
>> >> > > > > > > wrote:
>> >> > > > > > >
>> >> > > > > > >> If they don't find the time:
>> >> > > > > > >> They usually take the opposite path from me :D
>> >> > > > > > >> so the answer would be clear.
>> >> > > > > > >>
>> >> > > > > > >> hence my suggestion to vote.
>> >> > > > > > >>
>> >> > > > > > >>
>> >> > > > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
>> >> > > > > > >>> Hi Guozhang and Matthias
>> >> > > > > > >>>
>> >> > > > > > >>> I know both of you are quite busy, but we've gotten this
>> KIP
>> >> > to a
>> >> > > > > point
>> >> > > > > > >>> where we need more guidance on the API (perhaps a bit of
>> a
>> >> > > > > tie-breaker,
>> >> > > > > > >> if
>> >> > > > > > >>> you will). If you have anyone else you may think should
>> >> look at
>> >> > > > this,
>> >> > > > > > >>> please tag them accordingly.
>> >> > > > > > >>>
>> >> > > > > > >>> The scenario is as such:
>> >> > > > > > >>>
>> >> > > > > > >>> Current Option:
>> >> > > > > > >>> API:
>> >> > > > > > >>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>> >> > > > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
>> >> > > partition
>> >> > > > > with
>> >> > > > > > >> the
>> >> > > > > > >>> foreignKey (repartition 1)
>> >> > > > > > >>> 2) Join the data
>> >> > > > > > >>> 3) Shuffle the data back to the original node
>> (repartition
>> >> 2)
>> >> > > > > > >>> 4) Resolve out-of-order arrival / race condition due to
>> >> > > foreign-key
>> >> > > > > > >> changes.
>> >> > > > > > >>>
>> >> > > > > > >>> Alternate Option:
>> >> > > > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
>> >> > > > > > >>> - It would be keyed on a wrapped key function:
>> >> <CombinedKey<KO,
>> >> > > K>,
>> >> > > > > VR>
>> >> > > > > > >> (KO
>> >> > > > > > >>> = Other Table Key, K = This Table Key, VR = Joined
>> Result)
>> >> > > > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
>> >> > > otherwise a
>> >> > > > > > user
>> >> > > > > > >>> would be able to perform additional functions directly
>> from
>> >> the
>> >> > > > > > >>> KScatteredTable (TBD - currently out of scope).
>> >> > > > > > >>> - John's analysis 2-emails up is accurate as to the
>> >> tradeoffs.
>> >> > > > > > >>>
>> >> > > > > > >>> Current Option is coded as-is. Alternate option is
>> possible,
>> >> > but
>> >> > > > will
>> >> > > > > > >>> require for implementation details to be made in the API
>> and
>> >> > some
>> >> > > > > > >> exposure
>> >> > > > > > >>> of new data structures into the API (ie: CombinedKey).
>> >> > > > > > >>>
>> >> > > > > > >>> I appreciate any insight into this.
>> >> > > > > > >>>
>> >> > > > > > >>> Thanks.
>> >> > > > > > >>>
>> >> > > > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
>> >> > > > > > adam.bellemare@gmail.com>
>> >> > > > > > >>> wrote:
>> >> > > > > > >>>
>> >> > > > > > >>>> Hi John
>> >> > > > > > >>>>
>> >> > > > > > >>>> Thanks for your feedback and assistance. I think your
>> >> summary
>> >> > is
>> >> > > > > > >> accurate
>> >> > > > > > >>>> from my perspective. Additionally, I would like to add
>> that
>> >> > > there
>> >> > > > > is a
>> >> > > > > > >> risk
>> >> > > > > > >>>> of inconsistent final states without performing the
>> >> > resolution.
>> >> > > > This
>> >> > > > > > is
>> >> > > > > > >> a
>> >> > > > > > >>>> major concern for me as most of the data I have dealt
>> with
>> >> is
>> >> > > > > produced
>> >> > > > > > >> by
>> >> > > > > > >>>> relational databases. We have seen a number of cases
>> where
>> >> a
>> >> > > user
>> >> > > > in
>> >> > > > > > the
>> >> > > > > > >>>> Rails UI has modified the field (foreign key), realized
>> >> they
>> >> > > made
>> >> > > > a
>> >> > > > > > >>>> mistake, and then updated the field again with a new
>> key.
>> >> The
>> >> > > > events
>> >> > > > > > are
>> >> > > > > > >>>> propagated out as they are produced, and as such we have
>> >> had
>> >> > > > > > real-world
>> >> > > > > > >>>> cases where these inconsistencies were propagated
>> >> downstream
>> >> > as
>> >> > > > the
>> >> > > > > > >> final
>> >> > > > > > >>>> values due to the race conditions in the fanout of the
>> >> data.
>> >> > > > > > >>>>
>> >> > > > > > >>>> This solution that I propose values correctness of the
>> >> final
>> >> > > > result
>> >> > > > > > over
>> >> > > > > > >>>> other factors.
>> >> > > > > > >>>>
>> >> > > > > > >>>> We could always move this function over to using a
>> >> > > KScatteredTable
>> >> > > > > > >>>> implementation in the future, and simply deprecate it
>> this
>> >> > join
>> >> > > > API
>> >> > > > > in
>> >> > > > > > >>>> time. I think I would like to hear more from some of the
>> >> other
>> >> > > > major
>> >> > > > > > >>>> committers on which course of action they would think is
>> >> best
>> >> > > > before
>> >> > > > > > any
>> >> > > > > > >>>> more coding is done.
>> >> > > > > > >>>>
>> >> > > > > > >>>> Thanks again
>> >> > > > > > >>>>
>> >> > > > > > >>>> Adam
>> >> > > > > > >>>>
>> >> > > > > > >>>>
>> >> > > > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
>> >> > john@confluent.io>
>> >> > > > > > wrote:
>> >> > > > > > >>>>
>> >> > > > > > >>>>> Hi Jan and Adam,
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> Wow, thanks for doing that test, Adam. Those results
>> are
>> >> > > > > encouraging.
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> Thanks for your performance experience as well, Jan. I
>> >> agree
>> >> > > that
>> >> > > > > > >> avoiding
>> >> > > > > > >>>>> unnecessary join outputs is especially important when
>> the
>> >> > > fan-out
>> >> > > > > is
>> >> > > > > > so
>> >> > > > > > >>>>> high. I suppose this could also be built into the
>> >> > > implementation
>> >> > > > > > we're
>> >> > > > > > >>>>> discussing, but it wouldn't have to be specified in the
>> >> KIP
>> >> > > > (since
>> >> > > > > > >> it's an
>> >> > > > > > >>>>> API-transparent optimization).
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> As far as whether or not to re-repartition the data, I
>> >> didn't
>> >> > > > bring
>> >> > > > > > it
>> >> > > > > > >> up
>> >> > > > > > >>>>> because it sounded like the two of you agreed to leave
>> the
>> >> > KIP
>> >> > > > > as-is,
>> >> > > > > > >>>>> despite the disagreement.
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> If you want my opinion, I feel like both approaches are
>> >> > > > reasonable.
>> >> > > > > > >>>>> It sounds like Jan values more the potential for
>> >> developers
>> >> > to
>> >> > > > > > optimize
>> >> > > > > > >>>>> their topologies to re-use the intermediate nodes,
>> whereas
>> >> > Adam
>> >> > > > > > places
>> >> > > > > > >>>>> more
>> >> > > > > > >>>>> value on having a single operator that people can use
>> >> without
>> >> > > > extra
>> >> > > > > > >> steps
>> >> > > > > > >>>>> at the end.
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> Personally, although I do find it exceptionally
>> annoying
>> >> > when a
>> >> > > > > > >> framework
>> >> > > > > > >>>>> gets in my way when I'm trying to optimize something,
>> it
>> >> > seems
>> >> > > > > better
>> >> > > > > > >> to
>> >> > > > > > >>>>> go
>> >> > > > > > >>>>> for a single operation.
>> >> > > > > > >>>>> * Encapsulating the internal transitions gives us
>> >> significant
>> >> > > > > > latitude
>> >> > > > > > >> in
>> >> > > > > > >>>>> the implementation (for example, joining only at the
>> end,
>> >> not
>> >> > > in
>> >> > > > > the
>> >> > > > > > >>>>> middle
>> >> > > > > > >>>>> to avoid extra data copying and out-of-order
>> resolution;
>> >> how
>> >> > we
>> >> > > > > > >> represent
>> >> > > > > > >>>>> the first repartition keys (combined keys vs. value
>> >> vectors),
>> >> > > > > etc.).
>> >> > > > > > >> If we
>> >> > > > > > >>>>> publish something like a KScatteredTable with the
>> >> > > > right-partitioned
>> >> > > > > > >> joined
>> >> > > > > > >>>>> data, then the API pretty much locks in the
>> >> implementation as
>> >> > > > well.
>> >> > > > > > >>>>> * The API seems simpler to understand and use. I do
>> mean
>> >> > > "seems";
>> >> > > > > if
>> >> > > > > > >>>>> anyone
>> >> > > > > > >>>>> wants to make the case that KScatteredTable is actually
>> >> > > simpler,
>> >> > > > I
>> >> > > > > > >> think
>> >> > > > > > >>>>> hypothetical usage code would help. From a relational
>> >> algebra
>> >> > > > > > >> perspective,
>> >> > > > > > >>>>> it seems like KTable.join(KTable) should produce a new
>> >> KTable
>> >> > > in
>> >> > > > > all
>> >> > > > > > >>>>> cases.
>> >> > > > > > >>>>> * That said, there might still be room in the API for a
>> >> > > different
>> >> > > > > > >>>>> operation
>> >> > > > > > >>>>> like what Jan has proposed to scatter a KTable, and
>> then
>> >> do
>> >> > > > things
>> >> > > > > > like
>> >> > > > > > >>>>> join, re-group, etc from there... I'm not sure; I
>> haven't
>> >> > > thought
>> >> > > > > > >> through
>> >> > > > > > >>>>> all the consequences yet.
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> This is all just my opinion after thinking over the
>> >> > discussion
>> >> > > so
>> >> > > > > > >> far...
>> >> > > > > > >>>>> -John
>> >> > > > > > >>>>>
>> >> > > > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>> >> > > > > > >> adam.bellemare@gmail.com>
>> >> > > > > > >>>>> wrote:
>> >> > > > > > >>>>>
>> >> > > > > > >>>>>> Updated the PR to take into account John's feedback.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> I did some preliminary testing for the performance of
>> the
>> >> > > > > > prefixScan.
>> >> > > > > > >> I
>> >> > > > > > >>>>>> have attached the file, but I will also include the
>> text
>> >> in
>> >> > > the
>> >> > > > > body
>> >> > > > > > >>>>> here
>> >> > > > > > >>>>>> for archival purposes (I am not sure what happens to
>> >> > attached
>> >> > > > > > files).
>> >> > > > > > >> I
>> >> > > > > > >>>>>> also updated the PR and the KIP accordingly.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Summary: It scales exceptionally well for scanning
>> large
>> >> > > values
>> >> > > > of
>> >> > > > > > >>>>>> records. As Jan mentioned previously, the real issue
>> >> would
>> >> > be
>> >> > > > more
>> >> > > > > > >>>>> around
>> >> > > > > > >>>>>> processing the resulting records after obtaining them.
>> >> For
>> >> > > > > instance,
>> >> > > > > > >> it
>> >> > > > > > >>>>>> takes approximately ~80-120 mS to flush the buffer
>> and a
>> >> > > further
>> >> > > > > > >>>>> ~35-85mS
>> >> > > > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of
>> >> them.
>> >> > > > > Iterating
>> >> > > > > > >>>>>> through the records just to generate a simple count
>> >> takes ~
>> >> > 40
>> >> > > > > times
>> >> > > > > > >>>>> longer
>> >> > > > > > >>>>>> than the flush + scan combined.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> ============================================================================================
>> >> > > > > > >>>>>> Setup:
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> ============================================================================================
>> >> > > > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
>> >> > > (Xmx512m,
>> >> > > > > > >> Xms512m)
>> >> > > > > > >>>>>> CPU: i7 2.2 Ghz.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Note: I am using a slightly-modified,
>> directly-accessible
>> >> > > Kafka
>> >> > > > > > >> Streams
>> >> > > > > > >>>>>> RocksDB
>> >> > > > > > >>>>>> implementation (RocksDB.java, basically just avoiding
>> the
>> >> > > > > > >>>>>> ProcessorContext).
>> >> > > > > > >>>>>> There are no modifications to the default RocksDB
>> values
>> >> > > > provided
>> >> > > > > in
>> >> > > > > > >> the
>> >> > > > > > >>>>>> 2.1/trunk release.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> keysize = 128 bytes
>> >> > > > > > >>>>>> valsize = 512 bytes
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Step 1:
>> >> > > > > > >>>>>> Write X positive matching events: (key = prefix +
>> >> > left-padded
>> >> > > > > > >>>>>> auto-incrementing integer)
>> >> > > > > > >>>>>> Step 2:
>> >> > > > > > >>>>>> Write 10X negative matching events (key = left-padded
>> >> > > > > > >> auto-incrementing
>> >> > > > > > >>>>>> integer)
>> >> > > > > > >>>>>> Step 3:
>> >> > > > > > >>>>>> Perform flush
>> >> > > > > > >>>>>> Step 4:
>> >> > > > > > >>>>>> Perform prefixScan
>> >> > > > > > >>>>>> Step 5:
>> >> > > > > > >>>>>> Iterate through return Iterator and validate the
>> count of
>> >> > > > expected
>> >> > > > > > >>>>> events.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> ============================================================================================
>> >> > > > > > >>>>>> Results:
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> ============================================================================================
>> >> > > > > > >>>>>> X = 1k (11k events total)
>> >> > > > > > >>>>>> Flush Time = 39 mS
>> >> > > > > > >>>>>> Scan Time = 7 mS
>> >> > > > > > >>>>>> 6.9 MB disk
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> --------------------------------------------------------------------------------------------
>> >> > > > > > >>>>>> X = 10k (110k events total)
>> >> > > > > > >>>>>> Flush Time = 45 mS
>> >> > > > > > >>>>>> Scan Time = 8 mS
>> >> > > > > > >>>>>> 127 MB
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> --------------------------------------------------------------------------------------------
>> >> > > > > > >>>>>> X = 100k (1.1M events total)
>> >> > > > > > >>>>>> Test1:
>> >> > > > > > >>>>>> Flush Time = 60 mS
>> >> > > > > > >>>>>> Scan Time = 12 mS
>> >> > > > > > >>>>>> 678 MB
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Test2:
>> >> > > > > > >>>>>> Flush Time = 45 mS
>> >> > > > > > >>>>>> Scan Time = 7 mS
>> >> > > > > > >>>>>> 576 MB
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> --------------------------------------------------------------------------------------------
>> >> > > > > > >>>>>> X = 1MB (11M events total)
>> >> > > > > > >>>>>> Test1:
>> >> > > > > > >>>>>> Flush Time = 52 mS
>> >> > > > > > >>>>>> Scan Time = 19 mS
>> >> > > > > > >>>>>> 7.2 GB
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Test2:
>> >> > > > > > >>>>>> Flush Time = 84 mS
>> >> > > > > > >>>>>> Scan Time = 34 mS
>> >> > > > > > >>>>>> 9.1 GB
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> --------------------------------------------------------------------------------------------
>> >> > > > > > >>>>>> X = 2.5M (27.5M events total)
>> >> > > > > > >>>>>> Test1:
>> >> > > > > > >>>>>> Flush Time = 82 mS
>> >> > > > > > >>>>>> Scan Time = 63 mS
>> >> > > > > > >>>>>> 17GB - 276 sst files
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Test2:
>> >> > > > > > >>>>>> Flush Time = 116 mS
>> >> > > > > > >>>>>> Scan Time = 35 mS
>> >> > > > > > >>>>>> 23GB - 361 sst files
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Test3:
>> >> > > > > > >>>>>> Flush Time = 103 mS
>> >> > > > > > >>>>>> Scan Time = 82 mS
>> >> > > > > > >>>>>> 19 GB - 300 sst files
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> --------------------------------------------------------------------------------------------
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M
>> >> events. I
>> >> > > > tried
>> >> > > > > > to
>> >> > > > > > >> go
>> >> > > > > > >>>>>> to X = 10M (110M events) but RocksDB was going into
>> the
>> >> > 100GB+
>> >> > > > > range
>> >> > > > > > >>>>> and my
>> >> > > > > > >>>>>> laptop ran out of disk. More extensive testing could
>> be
>> >> done
>> >> > > > but I
>> >> > > > > > >>>>> suspect
>> >> > > > > > >>>>>> that it would be in line with what we're seeing in the
>> >> > results
>> >> > > > > > above.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> At this point in time, I think the only major
>> discussion
>> >> > point
>> >> > > > is
>> >> > > > > > >> really
>> >> > > > > > >>>>>> around what Jan and I have disagreed on:
>> repartitioning
>> >> > back +
>> >> > > > > > >> resolving
>> >> > > > > > >>>>>> potential out of order issues or leaving that up to
>> the
>> >> > client
>> >> > > > to
>> >> > > > > > >>>>> handle.
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Thanks folks,
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> Adam
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
>> >> > > > > > Jan.Filipiak@trivago.com
>> >> > > > > > >>>
>> >> > > > > > >>>>>> wrote:
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>> >> > > > > > >>>>>>>> Hi all,
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Sorry that this discussion petered out... I think
>> the
>> >> 2.1
>> >> > > > > release
>> >> > > > > > >>>>>>> caused an
>> >> > > > > > >>>>>>>> extended distraction that pushed it off everyone's
>> >> radar
>> >> > > > (which
>> >> > > > > > was
>> >> > > > > > >>>>>>>> precisely Adam's concern). Personally, I've also had
>> >> some
>> >> > > > extend
>> >> > > > > > >>>>>>>> distractions of my own that kept (and continue to
>> >> keep) me
>> >> > > > > > >>>>> preoccupied.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> However, calling for a vote did wake me up, so I
>> guess
>> >> Jan
>> >> > > was
>> >> > > > > on
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> right
>> >> > > > > > >>>>>>>> track!
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> I've gone back and reviewed the whole KIP document
>> and
>> >> the
>> >> > > > prior
>> >> > > > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> API Thoughts:
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 1. If I read the KIP right, you are proposing a
>> >> > many-to-one
>> >> > > > > join.
>> >> > > > > > >>>>> Could
>> >> > > > > > >>>>>>> we
>> >> > > > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
>> >> flip
>> >> > > the
>> >> > > > > > design
>> >> > > > > > >>>>>>> around
>> >> > > > > > >>>>>>>> and make it a oneToManyJoin?
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the
>> join
>> >> > > type,
>> >> > > > > and
>> >> > > > > > it
>> >> > > > > > >>>>>>> seems
>> >> > > > > > >>>>>>>> like it might trick some people into using it for a
>> >> > > one-to-one
>> >> > > > > > join.
>> >> > > > > > >>>>>>> This
>> >> > > > > > >>>>>>>> would work, of course, but it would be super
>> >> inefficient
>> >> > > > > compared
>> >> > > > > > to
>> >> > > > > > >>>>> a
>> >> > > > > > >>>>>>>> simple rekey-and-join.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 2. I might have missed it, but I don't think it's
>> >> > specified
>> >> > > > > > whether
>> >> > > > > > >>>>>>> it's an
>> >> > > > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer
>> >> join, as
>> >> > > > > > >>>>> (neglecting
>> >> > > > > > >>>>>>> IQ),
>> >> > > > > > >>>>>>>> the rest can be achieved by filtering or by handling
>> >> it in
>> >> > > the
>> >> > > > > > >>>>>>> ValueJoiner.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
>> quite
>> >> > > right.
>> >> > > > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
>> >> > > paradigms
>> >> > > > in
>> >> > > > > > >>>>> play in
>> >> > > > > > >>>>>>>> the Streams API, so it's confusing, but instead of
>> >> three
>> >> > > > > > Serialized
>> >> > > > > > >>>>>>> args, I
>> >> > > > > > >>>>>>>> think it would be better to have one that allows
>> >> > > (optionally)
>> >> > > > > > >> setting
>> >> > > > > > >>>>>>> the 4
>> >> > > > > > >>>>>>>> incoming serdes. The result serde is defined by the
>> >> > > > > Materialized.
>> >> > > > > > >> The
>> >> > > > > > >>>>>>>> incoming serdes can be optional because they might
>> >> already
>> >> > > be
>> >> > > > > > >>>>> available
>> >> > > > > > >>>>>>> on
>> >> > > > > > >>>>>>>> the source KTables, or the default serdes from the
>> >> config
>> >> > > > might
>> >> > > > > be
>> >> > > > > > >>>>>>>> applicable.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other
>> joins
>> >> > > don't
>> >> > > > > > allow
>> >> > > > > > >>>>>>> setting
>> >> > > > > > >>>>>>>> one, and it seems like it might actually be harmful,
>> >> since
>> >> > > the
>> >> > > > > > rekey
>> >> > > > > > >>>>>>>> operation needs to produce results that are
>> >> co-partitioned
>> >> > > > with
>> >> > > > > > the
>> >> > > > > > >>>>>>> "other"
>> >> > > > > > >>>>>>>> KTable.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I
>> >> didn't
>> >> > > > > actually
>> >> > > > > > >>>>>>> follow
>> >> > > > > > >>>>>>>> what Matthias meant about namespacing requiring
>> >> > > > "deserializing"
>> >> > > > > > the
>> >> > > > > > >>>>>>> record
>> >> > > > > > >>>>>>>> header. The headers are already Strings, so I don't
>> >> think
>> >> > > that
>> >> > > > > > >>>>>>>> deserialization is required. If we applied the
>> >> namespace
>> >> > at
>> >> > > > > source
>> >> > > > > > >>>>> nodes
>> >> > > > > > >>>>>>>> and stripped it at sink nodes, this would be
>> >> practically
>> >> > no
>> >> > > > > > >> overhead.
>> >> > > > > > >>>>>>> The
>> >> > > > > > >>>>>>>> advantage of the namespace idea is that no public
>> API
>> >> > change
>> >> > > > wrt
>> >> > > > > > >>>>> headers
>> >> > > > > > >>>>>>>> needs to happen, and no restrictions need to be
>> placed
>> >> on
>> >> > > > users'
>> >> > > > > > >>>>>>> headers.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> (Although I'm wondering if we can get away without
>> the
>> >> > > header
>> >> > > > at
>> >> > > > > > >>>>> all...
>> >> > > > > > >>>>>>>> stay tuned)
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM
>> >> table
>> >> > > > > growing
>> >> > > > > > >>>>>>> without
>> >> > > > > > >>>>>>>> bound. As I read it, the HWM table is effectively
>> >> > > implementing
>> >> > > > > OCC
>> >> > > > > > >> to
>> >> > > > > > >>>>>>>> resolve the problem you noted with disordering when
>> the
>> >> > > rekey
>> >> > > > is
>> >> > > > > > >>>>>>>> reversed... particularly notable when the FK
>> changes.
>> >> As
>> >> > > such,
>> >> > > > > it
>> >> > > > > > >>>>> only
>> >> > > > > > >>>>>>>> needs to track the most recent "version" (the
>> offset in
>> >> > the
>> >> > > > > source
>> >> > > > > > >>>>>>>> partition) of each key. Therefore, it should have
>> the
>> >> same
>> >> > > > > number
>> >> > > > > > of
>> >> > > > > > >>>>>>> keys
>> >> > > > > > >>>>>>>> as the source table at all times.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> I see that you are aware of KIP-258, which I think
>> >> might
>> >> > be
>> >> > > > > > relevant
>> >> > > > > > >>>>> in
>> >> > > > > > >>>>>>> a
>> >> > > > > > >>>>>>>> couple of ways. One: it's just about storing the
>> >> timestamp
>> >> > > in
>> >> > > > > the
>> >> > > > > > >>>>> state
>> >> > > > > > >>>>>>>> store, but the ultimate idea is to effectively use
>> the
>> >> > > > timestamp
>> >> > > > > > as
>> >> > > > > > >>>>> an
>> >> > > > > > >>>>>>> OCC
>> >> > > > > > >>>>>>>> "version" to drop disordered updates. You wouldn't
>> >> want to
>> >> > > use
>> >> > > > > the
>> >> > > > > > >>>>>>>> timestamp for this operation, but if you were to
>> use a
>> >> > > similar
>> >> > > > > > >>>>>>> mechanism to
>> >> > > > > > >>>>>>>> store the source offset in the store alongside the
>> >> > re-keyed
>> >> > > > > > values,
>> >> > > > > > >>>>> then
>> >> > > > > > >>>>>>>> you could avoid a separate table.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 6. You and Jan have been thinking about this for a
>> long
>> >> > > time,
>> >> > > > so
>> >> > > > > > >> I've
>> >> > > > > > >>>>>>>> probably missed something here, but I'm wondering
>> if we
>> >> > can
>> >> > > > > avoid
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> HWM
>> >> > > > > > >>>>>>>> tracking at all and resolve out-of-order during a
>> final
>> >> > join
>> >> > > > > > >>>>> instead...
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Let's say we're joining a left table (Integer K:
>> Letter
>> >> > FK,
>> >> > > > > (other
>> >> > > > > > >>>>>>> data))
>> >> > > > > > >>>>>>>> to a right table (Letter K: (some data)).
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Left table:
>> >> > > > > > >>>>>>>> 1: (A, xyz)
>> >> > > > > > >>>>>>>> 2: (B, asd)
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Right table:
>> >> > > > > > >>>>>>>> A: EntityA
>> >> > > > > > >>>>>>>> B: EntityB
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> We could do a rekey as you proposed with a combined
>> >> key,
>> >> > but
>> >> > > > not
>> >> > > > > > >>>>>>>> propagating the value at all..
>> >> > > > > > >>>>>>>> Rekey table:
>> >> > > > > > >>>>>>>> A-1: (dummy value)
>> >> > > > > > >>>>>>>> B-2: (dummy value)
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Which we then join with the right table to produce:
>> >> > > > > > >>>>>>>> A-1: EntityA
>> >> > > > > > >>>>>>>> B-2: EntityB
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Which gets rekeyed back:
>> >> > > > > > >>>>>>>> 1: A, EntityA
>> >> > > > > > >>>>>>>> 2: B, EntityB
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> And finally we do the actual join:
>> >> > > > > > >>>>>>>> Result table:
>> >> > > > > > >>>>>>>> 1: ((A, xyz), EntityA)
>> >> > > > > > >>>>>>>> 2: ((B, asd), EntityB)
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> The thing is that in that last join, we have the
>> >> > opportunity
>> >> > > > to
>> >> > > > > > >>>>> compare
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>> current FK in the left table with the incoming PK of
>> >> the
>> >> > > right
>> >> > > > > > >>>>> table. If
>> >> > > > > > >>>>>>>> they don't match, we just drop the event, since it
>> >> must be
>> >> > > > > > outdated.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>> In your KIP, you gave an example in which (1: A,
>> xyz)
>> >> gets
>> >> > > > > updated
>> >> > > > > > >> to
>> >> > > > > > >>>>>>> (1:
>> >> > > > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about
>> whether
>> >> the
>> >> > > > final
>> >> > > > > > >>>>> state
>> >> > > > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the
>> >> > algorithm
>> >> > > > > above,
>> >> > > > > > >>>>> you
>> >> > > > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
>> >> (B,
>> >> > > xyz),
>> >> > > > > (B,
>> >> > > > > > >>>>>>>> EntityB)). It seems like this does give you enough
>> >> > > information
>> >> > > > > to
>> >> > > > > > >>>>> make
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>> right choice, regardless of disordering.
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> Will check Adams patch, but this should work. As
>> >> mentioned
>> >> > > > often
>> >> > > > > I
>> >> > > > > > am
>> >> > > > > > >>>>>>> not convinced on partitioning back for the user
>> >> > > automatically.
>> >> > > > I
>> >> > > > > > >> think
>> >> > > > > > >>>>>>> this is the real performance eater ;)
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
>> >> > > > performance
>> >> > > > > of
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>> range scans when records change in the right table.
>> >> You've
>> >> > > > said
>> >> > > > > > that
>> >> > > > > > >>>>>>> you've
>> >> > > > > > >>>>>>>> been using the algorithm you presented in production
>> >> for a
>> >> > > > > while.
>> >> > > > > > >> Can
>> >> > > > > > >>>>>>> you
>> >> > > > > > >>>>>>>> give us a sense of the performance characteristics
>> >> you've
>> >> > > > > > observed?
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> Make it work, make it fast, make it beautiful. The
>> >> topmost
>> >> > > > thing
>> >> > > > > > here
>> >> > > > > > >>>>> is
>> >> > > > > > >>>>>>> / was correctness. In practice I do not measure the
>> >> > > performance
>> >> > > > > of
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> range scan. Usual cases I run this with is emitting
>> >> 500k -
>> >> > > 1kk
>> >> > > > > rows
>> >> > > > > > >>>>>>> on a left hand side change. The range scan is just
>> the
>> >> work
>> >> > > you
>> >> > > > > > gotta
>> >> > > > > > >>>>>>> do, also when you pack your data into different
>> formats,
>> >> > > > usually
>> >> > > > > > the
>> >> > > > > > >>>>>>> rocks performance is very tight to the size of the
>> data
>> >> and
>> >> > > we
>> >> > > > > > can't
>> >> > > > > > >>>>>>> really change that. It is more important for users to
>> >> > prevent
>> >> > > > > > useless
>> >> > > > > > >>>>>>> updates to begin with. My left hand side is guarded
>> to
>> >> drop
>> >> > > > > changes
>> >> > > > > > >>>>> that
>> >> > > > > > >>>>>>> are not going to change my join output.
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> usually it's:
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> drop unused fields and then don't forward if
>> >> > old.equals(new)
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>> regarding to the performance of creating an iterator
>> for
>> >> > > > smaller
>> >> > > > > > >>>>>>> fanouts, users can still just do a group by first
>> then
>> >> > > anyways.
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>>> I could only think of one alternative, but I'm not
>> >> sure if
>> >> > > > it's
>> >> > > > > > >>>>> better
>> >> > > > > > >>>>>>> or
>> >> > > > > > >>>>>>>> worse... If the first re-key only needs to preserve
>> the
>> >> > > > original
>> >> > > > > > >> key,
>> >> > > > > > >>>>>>> as I
>> >> > > > > > >>>>>>>> proposed in #6, then we could store a vector of
>> keys in
>> >> > the
>> >> > > > > value:
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Left table:
>> >> > > > > > >>>>>>>> 1: A,...
>> >> > > > > > >>>>>>>> 2: B,...
>> >> > > > > > >>>>>>>> 3: A,...
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Gets re-keyed:
>> >> > > > > > >>>>>>>> A: [1, 3]
>> >> > > > > > >>>>>>>> B: [2]
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Then, the rhs part of the join would only need a
>> >> regular
>> >> > > > > > single-key
>> >> > > > > > >>>>>>> lookup.
>> >> > > > > > >>>>>>>> Of course we have to deal with the problem of large
>> >> > values,
>> >> > > as
>> >> > > > > > >>>>> there's
>> >> > > > > > >>>>>>> no
>> >> > > > > > >>>>>>>> bound on the number of lhs records that can
>> reference
>> >> rhs
>> >> > > > > records.
>> >> > > > > > >>>>>>> Offhand,
>> >> > > > > > >>>>>>>> I'd say we could page the values, so when one row is
>> >> past
>> >> > > the
>> >> > > > > > >>>>>>> threshold, we
>> >> > > > > > >>>>>>>> append the key for the next page. Then in most
>> cases,
>> >> it
>> >> > > would
>> >> > > > > be
>> >> > > > > > a
>> >> > > > > > >>>>>>> single
>> >> > > > > > >>>>>>>> key lookup, but for large fan-out updates, it would
>> be
>> >> one
>> >> > > per
>> >> > > > > > (max
>> >> > > > > > >>>>>>> value
>> >> > > > > > >>>>>>>> size)/(avg lhs key size).
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> This seems more complex, though... Plus, I think
>> >> there's
>> >> > > some
>> >> > > > > > extra
>> >> > > > > > >>>>>>>> tracking we'd need to do to know when to emit a
>> >> > retraction.
>> >> > > > For
>> >> > > > > > >>>>> example,
>> >> > > > > > >>>>>>>> when record 1 is deleted, the re-key table would
>> just
>> >> have
>> >> > > (A:
>> >> > > > > > [3]).
>> >> > > > > > >>>>>>> Some
>> >> > > > > > >>>>>>>> kind of tombstone is needed so that the join result
>> >> for 1
>> >> > > can
>> >> > > > > also
>> >> > > > > > >> be
>> >> > > > > > >>>>>>>> retracted.
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> That's all!
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> Thanks so much to both Adam and Jan for the
>> thoughtful
>> >> > KIP.
>> >> > > > > Sorry
>> >> > > > > > >> the
>> >> > > > > > >>>>>>>> discussion has been slow.
>> >> > > > > > >>>>>>>> -John
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>> >> > > > > > >>>>> Jan.Filipiak@trivago.com>
>> >> > > > > > >>>>>>>> wrote:
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>>> Id say you can just call the vote.
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>>> that happens all the time, and if something comes
>> up,
>> >> it
>> >> > > just
>> >> > > > > > goes
>> >> > > > > > >>>>> back
>> >> > > > > > >>>>>>>>> to discuss.
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>>> would not expect to much attention with another
>> >> another
>> >> > > email
>> >> > > > > in
>> >> > > > > > >>>>> this
>> >> > > > > > >>>>>>>>> thread.
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>>> best Jan
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>> >> > > > > > >>>>>>>>>> Hello Contributors
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do
>> >> need
>> >> > to
>> >> > > > bump
>> >> > > > > > >>>>> this to
>> >> > > > > > >>>>>>>>> keep
>> >> > > > > > >>>>>>>>>> visibility up. I am still intending to push this
>> >> through
>> >> > > > once
>> >> > > > > > >>>>>>> contributor
>> >> > > > > > >>>>>>>>>> feedback is given.
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> Main points that need addressing:
>> >> > > > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
>> >> > > singular
>> >> > > > > > graph
>> >> > > > > > >>>>> node
>> >> > > > > > >>>>>>>>> into
>> >> > > > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters
>> right
>> >> > > now. I
>> >> > > > > am
>> >> > > > > > a
>> >> > > > > > >>>>> bit
>> >> > > > > > >>>>>>>>> fuzzy
>> >> > > > > > >>>>>>>>>> on how the optimizations are supposed to work, so
>> I
>> >> > would
>> >> > > > > > >>>>> appreciate
>> >> > > > > > >>>>>>> any
>> >> > > > > > >>>>>>>>>> help on this aspect.
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This
>> >> thread
>> >> > > has
>> >> > > > > > much
>> >> > > > > > >>>>>>>>> discourse
>> >> > > > > > >>>>>>>>>> between Jan and I between the current highwater
>> mark
>> >> > > > proposal
>> >> > > > > > and
>> >> > > > > > >> a
>> >> > > > > > >>>>>>>>> groupBy
>> >> > > > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we
>> need
>> >> to
>> >> > > > > strictly
>> >> > > > > > >>>>> handle
>> >> > > > > > >>>>>>>>> any
>> >> > > > > > >>>>>>>>>> chance of out-of-order data and leave none of it
>> up
>> >> to
>> >> > the
>> >> > > > > > >>>>> consumer.
>> >> > > > > > >>>>>>> Any
>> >> > > > > > >>>>>>>>>> comments or suggestions here would also help.
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> 3) Anything else that you see that would prevent
>> this
>> >> > from
>> >> > > > > > moving
>> >> > > > > > >>>>> to a
>> >> > > > > > >>>>>>>>> vote?
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> Thanks
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> Adam
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> >> > > > > > >>>>>>>>> adam.bellemare@gmail.com>
>> >> > > > > > >>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> Hi Jan
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
>> >> > > > > > >>>>> Stores.persistentWindowStore,
>> >> > > > > > >>>>>>> you
>> >> > > > > > >>>>>>>>>>> actually only need to specify the amount of
>> segments
>> >> > you
>> >> > > > want
>> >> > > > > > and
>> >> > > > > > >>>>> how
>> >> > > > > > >>>>>>>>> large
>> >> > > > > > >>>>>>>>>>> they are. To the best of my understanding, what
>> >> happens
>> >> > > is
>> >> > > > > that
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>>>> segments are automatically rolled over as new
>> data
>> >> with
>> >> > > new
>> >> > > > > > >>>>>>> timestamps
>> >> > > > > > >>>>>>>>> are
>> >> > > > > > >>>>>>>>>>> created. We use this exact functionality in some
>> of
>> >> the
>> >> > > > work
>> >> > > > > > done
>> >> > > > > > >>>>>>>>>>> internally at my company. For reference, this is
>> the
>> >> > > > hopping
>> >> > > > > > >>>>> windowed
>> >> > > > > > >>>>>>>>> store.
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> In the code that I have provided, there are going
>> >> to be
>> >> > > two
>> >> > > > > 24h
>> >> > > > > > >>>>>>>>> segments.
>> >> > > > > > >>>>>>>>>>> When a record is put into the windowStore, it
>> will
>> >> be
>> >> > > > > inserted
>> >> > > > > > at
>> >> > > > > > >>>>>>> time
>> >> > > > > > >>>>>>>>> T in
>> >> > > > > > >>>>>>>>>>> both segments. The two segments will always
>> overlap
>> >> by
>> >> > > 12h.
>> >> > > > > As
>> >> > > > > > >>>>> time
>> >> > > > > > >>>>>>>>> goes on
>> >> > > > > > >>>>>>>>>>> and new records are added (say at time T+12h+),
>> the
>> >> > > oldest
>> >> > > > > > >> segment
>> >> > > > > > >>>>>>> will
>> >> > > > > > >>>>>>>>> be
>> >> > > > > > >>>>>>>>>>> automatically deleted and a new segment created.
>> The
>> >> > > > records
>> >> > > > > > are
>> >> > > > > > >>>>> by
>> >> > > > > > >>>>>>>>> default
>> >> > > > > > >>>>>>>>>>> inserted with the context.timestamp(), such that
>> it
>> >> is
>> >> > > the
>> >> > > > > > record
>> >> > > > > > >>>>>>> time,
>> >> > > > > > >>>>>>>>> not
>> >> > > > > > >>>>>>>>>>> the clock time, which is used.
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> To the best of my understanding, the timestamps
>> are
>> >> > > > retained
>> >> > > > > > when
>> >> > > > > > >>>>>>>>>>> restoring from the changelog.
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with
>> TTL
>> >> > at a
>> >> > > > > > >>>>>>> segment-level,
>> >> > > > > > >>>>>>>>>>> instead of at an individual record level.
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>> >> > > > > > >>>>>>> Jan.Filipiak@trivago.com>
>> >> > > > > > >>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
>> >> > > > > > ClassCastException
>> >> > > > > > >>>>> or
>> >> > > > > > >>>>>>>>>>>> similar.
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> You either would have to specify the window you
>> >> > > fetch/put
>> >> > > > or
>> >> > > > > > >>>>> iterate
>> >> > > > > > >>>>>>>>>>>> across all windows the key was found in right?
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> I just hope the window-store doesn't check
>> >> stream-time
>> >> > > > under
>> >> > > > > > the
>> >> > > > > > >>>>>>> hoods
>> >> > > > > > >>>>>>>>>>>> that would be a questionable interface.
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> If it does: did you see my comment on checking
>> all
>> >> the
>> >> > > > > windows
>> >> > > > > > >>>>>>> earlier?
>> >> > > > > > >>>>>>>>>>>> that would be needed to actually give reasonable
>> >> time
>> >> > > > > > gurantees.
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> Best
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> >> > > > > > >>>>>>>>>>>>> Hi Jan
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
>> >> changed
>> >> > > the
>> >> > > > > > state
>> >> > > > > > >>>>>>> store,
>> >> > > > > > >>>>>>>>>>>> not
>> >> > > > > > >>>>>>>>>>>>> the ProcessorSupplier.
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>> Thanks,
>> >> > > > > > >>>>>>>>>>>>> Adam
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> @Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed
>> >> > something
>> >> > > > that
>> >> > > > > > >>>>> will be
>> >> > > > > > >>>>>>>>>>>>>>> extremely
>> >> > > > > > >>>>>>>>>>>>>>> useful for this KIP.
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> @Jan
>> >> > > > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being
>> said, I
>> >> > will
>> >> > > > not
>> >> > > > > > be
>> >> > > > > > >>>>>>> moving
>> >> > > > > > >>>>>>>>>>>> ahead
>> >> > > > > > >>>>>>>>>>>>>>> with an implementation using
>> reshuffle/groupBy
>> >> > > solution
>> >> > > > > as
>> >> > > > > > >> you
>> >> > > > > > >>>>>>>>>>>> propose.
>> >> > > > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it
>> >> > yourself
>> >> > > > off
>> >> > > > > > of
>> >> > > > > > >>>>> my
>> >> > > > > > >>>>>>>>>>>> current PR
>> >> > > > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I
>> >> would
>> >> > > be
>> >> > > > > more
>> >> > > > > > >>>>> than
>> >> > > > > > >>>>>>>>>>>> happy to
>> >> > > > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it
>> >> > stands
>> >> > > > > right
>> >> > > > > > >>>>> now,
>> >> > > > > > >>>>>>> I do
>> >> > > > > > >>>>>>>>>>>> not
>> >> > > > > > >>>>>>>>>>>>>>> really have more time to invest into
>> >> alternatives
>> >> > > > without
>> >> > > > > > >>>>> there
>> >> > > > > > >>>>>>>>> being
>> >> > > > > > >>>>>>>>>>>> a
>> >> > > > > > >>>>>>>>>>>>>>> strong indication from the binding voters
>> which
>> >> > they
>> >> > > > > would
>> >> > > > > > >>>>>>> prefer.
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally
>> gave
>> >> up
>> >> > on
>> >> > > > the
>> >> > > > > > >>>>> streams
>> >> > > > > > >>>>>>>>> DSL
>> >> > > > > > >>>>>>>>>>>> for
>> >> > > > > > >>>>>>>>>>>>>> some time already, otherwise I would have
>> pulled
>> >> > this
>> >> > > > KIP
>> >> > > > > > >>>>> through
>> >> > > > > > >>>>>>>>>>>> already.
>> >> > > > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL
>> based on
>> >> > > PAPI.
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the
>> >> windowed
>> >> > > > state
>> >> > > > > > >>>>> store
>> >> > > > > > >>>>>>> in
>> >> > > > > > >>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>> next
>> >> > > > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and
>> then I
>> >> > will
>> >> > > > come
>> >> > > > > > >> back
>> >> > > > > > >>>>>>> for
>> >> > > > > > >>>>>>>>>>>> final
>> >> > > > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that
>> any of
>> >> > the
>> >> > > > > > binding
>> >> > > > > > >>>>>>> voters
>> >> > > > > > >>>>>>>>>>>> could
>> >> > > > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
>> >> updated
>> >> > it
>> >> > > > > > >>>>> according
>> >> > > > > > >>>>>>> to
>> >> > > > > > >>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>> latest plan:
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> >> > > > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a
>> windowed
>> >> > > store.
>> >> > > > > > This
>> >> > > > > > >>>>>>> could
>> >> > > > > > >>>>>>>>> be
>> >> > > > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
>> they
>> >> > are
>> >> > > > > > >>>>> completed.
>> >> > > > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> Thanks,
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> Adam
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
>> >> > already
>> >> > > > > > updated
>> >> > > > > > >>>>> in
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> PR?
>> >> > > > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
>> Missing
>> >> > > > > something?
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
>> Wang <
>> >> > > > > > >>>>>>> wangguoz@gmail.com>
>> >> > > > > > >>>>>>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533
>> is
>> >> the
>> >> > > > wrong
>> >> > > > > > >> link,
>> >> > > > > > >>>>>>> as it
>> >> > > > > > >>>>>>>>>>>> is
>> >> > > > > > >>>>>>>>>>>>>>>> for
>> >> > > > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
>> >> part of
>> >> > > > > KIP-258
>> >> > > > > > >>>>> we do
>> >> > > > > > >>>>>>>>>>>> want to
>> >> > > > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source
>> >> > KTable"
>> >> > > > such
>> >> > > > > > >> that
>> >> > > > > > >>>>>>>>>>>> instead of
>> >> > > > > > >>>>>>>>>>>>>>>> blindly apply the updates to the
>> materialized
>> >> > store,
>> >> > > > > i.e.
>> >> > > > > > >>>>>>> following
>> >> > > > > > >>>>>>>>>>>>>>>> offset
>> >> > > > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are
>> older
>> >> > than
>> >> > > > the
>> >> > > > > > >>>>> current
>> >> > > > > > >>>>>>>>>>>> key's
>> >> > > > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
>> ordering.
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>> Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
>> >> Wang <
>> >> > > > > > >>>>>>>>> wangguoz@gmail.com>
>> >> > > > > > >>>>>>>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>> Hello Adam,
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
>> >> final
>> >> > > step
>> >> > > > > > (i.e.
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>> high
>> >> > > > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
>> >> with
>> >> > a
>> >> > > > > window
>> >> > > > > > >>>>>>> store),
>> >> > > > > > >>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>> think
>> >> > > > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually
>> >> help:
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a
>> >> key-value
>> >> > > > store
>> >> > > > > > >>>>> (i.e.
>> >> > > > > > >>>>>>> only
>> >> > > > > > >>>>>>>>>>>> for
>> >> > > > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
>> >> usage,
>> >> > as
>> >> > > > > > >>>>> described
>> >> > > > > > >>>>>>> in
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> https://issues.apache.org/jira/browse/KAFKA-5533
>> >> > ,
>> >> > > is
>> >> > > > > > that
>> >> > > > > > >>>>> we
>> >> > > > > > >>>>>>> can
>> >> > > > > > >>>>>>>>>>>> then
>> >> > > > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if
>> its
>> >> > > > > timestamp
>> >> > > > > > is
>> >> > > > > > >>>>>>>>> smaller
>> >> > > > > > >>>>>>>>>>>> than
>> >> > > > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp.
>> I
>> >> > think
>> >> > > it
>> >> > > > > is
>> >> > > > > > >>>>> very
>> >> > > > > > >>>>>>>>>>>> similar to
>> >> > > > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark
>> based
>> >> > > > > filtering,
>> >> > > > > > >>>>> while
>> >> > > > > > >>>>>>>>> you
>> >> > > > > > >>>>>>>>>>>> only
>> >> > > > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of
>> the
>> >> > > joining
>> >> > > > > > >> records
>> >> > > > > > >>>>>>> are
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>> correctly
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the
>> >> final
>> >> > > > stage.
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
>> and
>> >> > hence
>> >> > > > > > >>>>>>> non-windowed
>> >> > > > > > >>>>>>>>>>>> KTables
>> >> > > > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
>> >> really
>> >> > > have
>> >> > > > a
>> >> > > > > > good
>> >> > > > > > >>>>>>>>> support
>> >> > > > > > >>>>>>>>>>>> for
>> >> > > > > > >>>>>>>>>>>>>>>>> their joins anyways (
>> >> > > > > > >>>>>>>>>>>>
>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> >> > > > > > >>>>>>>>>>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
>> >> > > KTable-KTable
>> >> > > > > > >>>>> non-key
>> >> > > > > > >>>>>>>>> joins
>> >> > > > > > >>>>>>>>>>>> for
>> >> > > > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
>> Filipiak
>> >> <
>> >> > > > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation
>> would
>> >> > grow
>> >> > > > > > >> endlessly
>> >> > > > > > >>>>>>> based
>> >> > > > > > >>>>>>>>>>>> on
>> >> > > > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a
>> pair
>> >> of
>> >> > > > (<this
>> >> > > > > > >>>>> table
>> >> > > > > > >>>>>>>>>>>> primary
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> key>,
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This
>> is
>> >> used
>> >> > > to
>> >> > > > > > >>>>>>> differentiate
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> between
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
>> >> proposal
>> >> > > > would
>> >> > > > > > be
>> >> > > > > > >>>>> to
>> >> > > > > > >>>>>>>>>>>> replace
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> it
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
>> >> This
>> >> > > would
>> >> > > > > > allow
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>> same
>> >> > > > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on
>> time.
>> >> This
>> >> > > > > should
>> >> > > > > > >>>>> allow
>> >> > > > > > >>>>>>> for
>> >> > > > > > >>>>>>>>>>>> all
>> >> > > > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
>> >> > should
>> >> > > be
>> >> > > > > > >>>>>>> customizable
>> >> > > > > > >>>>>>>>>>>> by
>> >> > > > > > >>>>>>>>>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
>> >> perhaps
>> >> > > just
>> >> > > > > 10
>> >> > > > > > >>>>>>> minutes
>> >> > > > > > >>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> window,
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can
>> do
>> >> the
>> >> > > > trick
>> >> > > > > > >> here.
>> >> > > > > > >>>>>>> Even
>> >> > > > > > >>>>>>>>>>>> if I
>> >> > > > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
>> >> > > repartitioning
>> >> > > > > > >>>>> optional
>> >> > > > > > >>>>>>>>>>>> since I
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
>> >> am a
>> >> > > > little
>> >> > > > > > bit
>> >> > > > > > >>>>>>>>>>>> sceptical
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> about
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially
>> one
>> >> > > could
>> >> > > > > run
>> >> > > > > > >>>>> into
>> >> > > > > > >>>>>>>>>>>> problems
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> when
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window
>> >> border. I
>> >> > > will
>> >> > > > > > check
>> >> > > > > > >>>>> you
>> >> > > > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its
>> >> problematic, we
>> >> > > > could
>> >> > > > > > >>>>> still
>> >> > > > > > >>>>>>>>> check
>> >> > > > > > >>>>>>>>>>>>>>>>>> _all_
>> >> > > > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad
>> performance
>> >> > > impact I
>> >> > > > > > >> guess.
>> >> > > > > > >>>>>>> Will
>> >> > > > > > >>>>>>>>>>>> let
>> >> > > > > > >>>>>>>>>>>>>>>>>> you
>> >> > > > > > >>>>>>>>>>>>>>>>>> know if the implementation would be
>> correct
>> >> as
>> >> > > is. I
>> >> > > > > > >>>>> wouldn't
>> >> > > > > > >>>>>>> not
>> >> > > > > > >>>>>>>>>>>> like
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
>> >> > > timestamp(A)  <
>> >> > > > > > >>>>>>>>> timestamp(B).
>> >> > > > > > >>>>>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> think
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> we can't expect that.
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> @Jan
>> >> > > > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now
>> -
>> >> > thanks
>> >> > > > for
>> >> > > > > > the
>> >> > > > > > >>>>>>>>>>>> diagram, it
>> >> > > > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I
>> do
>> >> not
>> >> > > have
>> >> > > > > the
>> >> > > > > > >>>>>>> original
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> primary
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
>> >> > > available
>> >> > > > > > then
>> >> > > > > > >>>>> you
>> >> > > > > > >>>>>>>>>>>> would be
>> >> > > > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the
>> Map.
>> >> > That
>> >> > > > > being
>> >> > > > > > >>>>> said,
>> >> > > > > > >>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> encourage
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
>> for
>> >> > > clarity
>> >> > > > > for
>> >> > > > > > >>>>>>> everyone
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> else.
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just
>> really
>> >> hard
>> >> > > > work.
>> >> > > > > > But
>> >> > > > > > >>>>> I
>> >> > > > > > >>>>>>>>>>>> understand
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
>> >> > > original
>> >> > > > > > >> primary
>> >> > > > > > >>>>>>> key,
>> >> > > > > > >>>>>>>>> We
>> >> > > > > > >>>>>>>>>>>>>>>>>> have
>> >> > > > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in
>> PAPI
>> >> > and
>> >> > > > > > >> basically
>> >> > > > > > >>>>>>> not
>> >> > > > > > >>>>>>>>>>>> using
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> any
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
>> missed
>> >> > that
>> >> > > in
>> >> > > > > > >>>>> original
>> >> > > > > > >>>>>>> DSL
>> >> > > > > > >>>>>>>>>>>> its
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> not
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
>> >> up on
>> >> > > my
>> >> > > > > end.
>> >> > > > > > >>>>> Will
>> >> > > > > > >>>>>>>>>>>> finish
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
>> >> week.
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't
>> the
>> >> Map
>> >> > > stay
>> >> > > > > > >> inside
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> State
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
>> changes
>> >> > have
>> >> > > > > > >>>>> propagated?
>> >> > > > > > >>>>>>>>> Isn't
>> >> > > > > > >>>>>>>>>>>>>>>>>>> this
>> >> > > > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
>> >> state
>> >> > > > store?
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
>> >> substractor
>> >> > is
>> >> > > > > gonna
>> >> > > > > > >>>>>>> return
>> >> > > > > > >>>>>>>>>>>> `null`
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
>> >> there
>> >> > is
>> >> > > > > going
>> >> > > > > > to
>> >> > > > > > >>>>> be
>> >> > > > > > >>>>>>> a
>> >> > > > > > >>>>>>>>>>>> store
>> >> > > > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use
>> this
>> >> > store
>> >> > > > > > directly
>> >> > > > > > >>>>> for
>> >> > > > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
>> is a
>> >> > > > regular
>> >> > > > > > >>>>> store,
>> >> > > > > > >>>>>>>>>>>> satisfying
>> >> > > > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
>> >> join.
>> >> > > The
>> >> > > > > > >>>>> Windowed
>> >> > > > > > >>>>>>>>>>>> store is
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> not
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next
>> statefull
>> >> > > > operation
>> >> > > > > > we
>> >> > > > > > >>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
>> >> have
>> >> > the
>> >> > > > > > window
>> >> > > > > > >>>>>>> store
>> >> > > > > > >>>>>>>>>>>> also
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> have
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> the values then.
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
>> custom
>> >> > group
>> >> > > > by
>> >> > > > > > >>>>> before
>> >> > > > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary
>> key i
>> >> > think
>> >> > > > it
>> >> > > > > > >> would
>> >> > > > > > >>>>>>> help
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> users
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given
>> the
>> >> > > > original
>> >> > > > > > >>>>> primary
>> >> > > > > > >>>>>>>>> key
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> issue I
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid
>> >> foundation
>> >> > > to
>> >> > > > > > build
>> >> > > > > > >>>>> on.
>> >> > > > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
>> user.
>> >> > very
>> >> > > > > > >>>>>>> unfortunate. I
>> >> > > > > > >>>>>>>>>>>> could
>> >> > > > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I
>> do
>> >> not
>> >> > > > think
>> >> > > > > > its
>> >> > > > > > >>>>> a
>> >> > > > > > >>>>>>> good
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> decision.
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Thanks
>> >> > > > > > >>>>>>>>>>>>>>>>>>> Adam
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM,
>> Prajakta
>> >> > > Dumbre <
>> >> > > > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>> >> > > > > > >>>>>>> dumbreprajakta311@gmail.com
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           please remove me from this
>> group
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM
>> >> Jan
>> >> > > > > Filipiak
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> >> <mailto:
>> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make
>> >> such a
>> >> > > > > chart.
>> >> > > > > > >> last
>> >> > > > > > >>>>>>> time i
>> >> > > > > > >>>>>>>>>>>> didn't
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           get along
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined
>> all
>> >> your
>> >> > > > > charts.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done
>> >> today
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam
>> >> Bellemare
>> >> > > > > wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram
>> of
>> >> > what I
>> >> > > > > > >> attempted
>> >> > > > > > >>>>> on
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> KIP.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>
>> >> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> >> > > > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>
>> >> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> >> > > > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at
>> the
>> >> > start
>> >> > > of
>> >> > > > > my
>> >> > > > > > own
>> >> > > > > > >>>>>>>>>>>> implementation
>> >> > > > > > >>>>>>>>>>>>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           this
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could
>> >> not
>> >> > get
>> >> > > > it
>> >> > > > > to
>> >> > > > > > >>>>> work I
>> >> > > > > > >>>>>>> have
>> >> > > > > > >>>>>>>>>>>> since
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           discarded the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in
>> time,
>> >> if
>> >> > you
>> >> > > > > wish
>> >> > > > > > to
>> >> > > > > > >>>>>>> continue
>> >> > > > > > >>>>>>>>>>>> pursuing
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           for your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask
>> that
>> >> you
>> >> > > > please
>> >> > > > > > >>>>> create a
>> >> > > > > > >>>>>>>>>>>> diagram on
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           the KIP
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your
>> >> > solution.
>> >> > > > > > Please
>> >> > > > > > >>>>> feel
>> >> > > > > > >>>>>>> free
>> >> > > > > > >>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>> use
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           the image I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting
>> >> point.
>> >> > I
>> >> > > am
>> >> > > > > > having
>> >> > > > > > >>>>>>> trouble
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           understanding your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think
>> that
>> >> a
>> >> > > > > carefully
>> >> > > > > > >>>>>>> constructed
>> >> > > > > > >>>>>>>>>>>> diagram
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           will clear
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > up
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings.
>> >> > Alternately,
>> >> > > > > > please
>> >> > > > > > >>>>> post a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only
>> >> guess
>> >> > at
>> >> > > > what
>> >> > > > > > you
>> >> > > > > > >>>>>>> mean, and
>> >> > > > > > >>>>>>>>>>>> since I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           value my
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > own
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value
>> >> yours,
>> >> > I
>> >> > > > > > believe
>> >> > > > > > >> it
>> >> > > > > > >>>>> is
>> >> > > > > > >>>>>>> your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           responsibility to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation
>> >> instead
>> >> > > of
>> >> > > > me
>> >> > > > > > >>>>> trying to
>> >> > > > > > >>>>>>>>> guess.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > Adam
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00
>> >> AM,
>> >> > Jan
>> >> > > > > > Filipiak
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> >> <mailto:
>> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing
>> >> > interested.
>> >> > > > > kafka
>> >> > > > > > >>>>>>> streams at
>> >> > > > > > >>>>>>>>>>>> this
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           point supports
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as
>> long as
>> >> > both
>> >> > > > > > streams
>> >> > > > > > >>>>> have
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> same
>> >> > > > > > >>>>>>>>>>>>>>>>>>> key.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently
>> >> implementing a
>> >> > > > join
>> >> > > > > > >> where a
>> >> > > > > > >>>>>>> KTable
>> >> > > > > > >>>>>>>>>>>> and a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           KTable can
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation
>> ship
>> >> > (1:n).
>> >> > > > We
>> >> > > > > > >> exploit
>> >> > > > > > >>>>>>> that
>> >> > > > > > >>>>>>>>>>>> rocksdb
>> >> > > > > > >>>>>>>>>>>>>>>>>>> is
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> a
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data
>> >> sorted
>> >> > (At
>> >> > > > > least
>> >> > > > > > >>>>>>> exposes an
>> >> > > > > > >>>>>>>>>>>> API to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           access the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted
>> >> > fashion).
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical
>> caveats
>> >> > are
>> >> > > > well
>> >> > > > > > >>>>>>> understood
>> >> > > > > > >>>>>>>>> now
>> >> > > > > > >>>>>>>>>>>> and we
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> are
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > basically
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API
>> >> > Design
>> >> > > (
>> >> > > > > when
>> >> > > > > > >> Adam
>> >> > > > > > >>>>>>> sees
>> >> > > > > > >>>>>>>>> my
>> >> > > > > > >>>>>>>>>>>> newest
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           message).
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track
>> >> record of
>> >> > > > > loosing
>> >> > > > > > >>>>> those
>> >> > > > > > >>>>>>> kinda
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           arguments within
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I
>> have
>> >> no
>> >> > > clue
>> >> > > > > > why.
>> >> > > > > > >> So
>> >> > > > > > >>>>> I
>> >> > > > > > >>>>>>>>>>>> literally
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           can't wait for
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > you
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this
>> thread
>> >> and
>> >> > > > give
>> >> > > > > > you
>> >> > > > > > >>>>>>> opinion on
>> >> > > > > > >>>>>>>>>>>> how we
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           should
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > design
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
>> >> > > oneToManyJoin
>> >> > > > > and
>> >> > > > > > >> how
>> >> > > > > > >>>>>>> many
>> >> > > > > > >>>>>>>>>>>> power we
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           want to give
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity"
>> >> (where
>> >> > > > > > simplicity
>> >> > > > > > >>>>> isn't
>> >> > > > > > >>>>>>>>>>>> really that
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           as users
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > still
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I
>> >> argue)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join
>> in on
>> >> > the
>> >> > > > > > >> discussion
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James
>> >> Kwan
>> >> > > > wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group
>> and I
>> >> > > found
>> >> > > > > this
>> >> > > > > > >>>>> subject
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > like
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to
>> implement a
>> >> > join
>> >> > > > > > table of
>> >> > > > > > >>>>> two
>> >> > > > > > >>>>>>>>>>>> streams? Is
>> >> > > > > > >>>>>>>>>>>>>>>>>>> there
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > somewhere
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
>> >> > > requirement
>> >> > > > or
>> >> > > > > > >>>>> proposal?
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13
>> AM,
>> >> Jan
>> >> > > > > > Filipiak
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> >> <mailto:
>> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17,
>> Adam
>> >> > > > Bellemare
>> >> > > > > > >> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing
>> >> using a
>> >> > > > > > Windowed
>> >> > > > > > >>>>> Store
>> >> > > > > > >>>>>>> to
>> >> > > > > > >>>>>>>>>>>> store the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           highwater
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this
>> >> > should
>> >> > > > work
>> >> > > > > > >> fine,
>> >> > > > > > >>>>>>> with
>> >> > > > > > >>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>> caveat
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           being that
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > it
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve
>> out-of-order
>> >> > > arrival
>> >> > > > > > for up
>> >> > > > > > >>>>> to
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> size of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           the window
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > (ie:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This
>> would
>> >> > > remove
>> >> > > > > the
>> >> > > > > > >>>>>>> possibility
>> >> > > > > > >>>>>>>>>>>> of it
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> being
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > unbounded
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
>> >> > > > suggestion, I
>> >> > > > > > >>>>> believe
>> >> > > > > > >>>>>>> this
>> >> > > > > > >>>>>>>>> is
>> >> > > > > > >>>>>>>>>>>> where
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           we will
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement.
>> >> > While I
>> >> > > > do
>> >> > > > > > not
>> >> > > > > > >>>>>>> disagree
>> >> > > > > > >>>>>>>>>>>> with your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           statement
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be
>> >> additional
>> >> > > > joins
>> >> > > > > > done
>> >> > > > > > >>>>> in a
>> >> > > > > > >>>>>>>>>>>> real-world
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           workflow, I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > do
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can
>> >> conclusively
>> >> > > deal
>> >> > > > > > with
>> >> > > > > > >>>>>>>>> out-of-order
>> >> > > > > > >>>>>>>>>>>>>>>>>>> arrival
>> >> > > > > > >>>>>>>>>>>>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent
>> >> > joins. I
>> >> > > > > have
>> >> > > > > > >>>>>>> attempted
>> >> > > > > > >>>>>>>>> what
>> >> > > > > > >>>>>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           think you have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a
>> >> > high-water,
>> >> > > > > using
>> >> > > > > > >>>>>>> groupBy and
>> >> > > > > > >>>>>>>>>>>> reduce)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           and found
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes
>> >> too
>> >> > > > > quickly,
>> >> > > > > > or
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>> load
>> >> > > > > > >>>>>>>>> on
>> >> > > > > > >>>>>>>>>>>> a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           stream thread
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > is
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined
>> messages
>> >> > will
>> >> > > > > arrive
>> >> > > > > > >>>>>>>>> out-of-order
>> >> > > > > > >>>>>>>>>>>> and be
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           incorrectly
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that
>> an
>> >> > > > > intermediate
>> >> > > > > > >>>>> event
>> >> > > > > > >>>>>>> is
>> >> > > > > > >>>>>>>>>>>>>>>>>>> represented
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           as the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > final
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light
>> on
>> >> > your
>> >> > > > > > groupBy
>> >> > > > > > >>>>>>>>>>>> implementation.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           There must be
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion
>> where it
>> >> > is,
>> >> > > I
>> >> > > > > > would
>> >> > > > > > >>>>> just
>> >> > > > > > >>>>>>> like
>> >> > > > > > >>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it
>> >> must be
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess
>> up.
>> >> I
>> >> > > would
>> >> > > > > > like
>> >> > > > > > >> to
>> >> > > > > > >>>>>>> clarify
>> >> > > > > > >>>>>>>>>>>> before
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           we draw a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the
>> >> > scattered
>> >> > > > > events
>> >> > > > > > >>>>> back to
>> >> > > > > > >>>>>>>>> their
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> original
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only
>> way I
>> >> > know
>> >> > > > how
>> >> > > > > > to
>> >> > > > > > >>>>>>>>> conclusively
>> >> > > > > > >>>>>>>>>>>> deal
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           with
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in
>> a
>> >> > given
>> >> > > > time
>> >> > > > > > >> frame,
>> >> > > > > > >>>>>>> and to
>> >> > > > > > >>>>>>>>>>>> ensure
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           that the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > data
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent
>> with
>> >> > the
>> >> > > > > input
>> >> > > > > > >>>>> events.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code
>> to
>> >> > share
>> >> > > > that
>> >> > > > > > >>>>>>> illustrates
>> >> > > > > > >>>>>>>>> your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           approach, I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > would
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it
>> would
>> >> > > remove
>> >> > > > > any
>> >> > > > > > >>>>>>>>>>>> misunderstandings
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           that I may
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking
>> >> for
>> >> > my
>> >> > > > > code.
>> >> > > > > > I
>> >> > > > > > >>>>> don't
>> >> > > > > > >>>>>>> have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           something easily
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its
>> >> bloated
>> >> > > with
>> >> > > > > > >>>>> OO-patterns.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K
>> >> aggKey,
>> >> > V
>> >> > > > > > value, T
>> >> > > > > > >>>>>>>>> aggregate)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
>> >> > > > > currentStateAsMap =
>> >> > > > > > >>>>>>>>>>>> asMap(aggregate);
>> >> > > > > > >>>>>>>>>>>>>>>>>>> <<
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           imaginary
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
>> >> > > > > > >>>>> mapper.apply(value);
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is
>> the
>> >> > > place
>> >> > > > > > where
>> >> > > > > > >>>>> people
>> >> > > > > > >>>>>>>>>>>> actually
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           gonna have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > issues
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably
>> >> couldn't
>> >> > do
>> >> > > > it.
>> >> > > > > > we
>> >> > > > > > >>>>> would
>> >> > > > > > >>>>>>> need
>> >> > > > > > >>>>>>>>>>>> to find
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           a solution
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > here.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that
>> yet.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we
>> >> propagate
>> >> > the
>> >> > > > > > field in
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>>>>> joiner, so
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           that we can
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > pick
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate.
>> >> > Probably
>> >> > > > you
>> >> > > > > > have
>> >> > > > > > >>>>> not
>> >> > > > > > >>>>>>>>> thought
>> >> > > > > > >>>>>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           this in your
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am
>> very
>> >> open
>> >> > > to
>> >> > > > > > find a
>> >> > > > > > >>>>>>> generic
>> >> > > > > > >>>>>>>>>>>> solution
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           here. In my
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is
>> >> broken
>> >> > in
>> >> > > > > > >>>>>>>>> KTableImpl.GroupBy
>> >> > > > > > >>>>>>>>>>>> that
>> >> > > > > > >>>>>>>>>>>>>>>>>>> it
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           looses
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the keys
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the
>> >> > aggregate
>> >> > > > > key.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I
>> >> abstracted
>> >> > it
>> >> > > > away
>> >> > > > > > back
>> >> > > > > > >>>>>>> then way
>> >> > > > > > >>>>>>>>>>>> before
>> >> > > > > > >>>>>>>>>>>>>>>>>>> i
>> >> > > > > > >>>>>>>>>>>>>>>>>>> was
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > thinking
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That
>> is
>> >> > why I
>> >> > > > > > didn't
>> >> > > > > > >>>>>>> realize
>> >> > > > > > >>>>>>>>> its
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           significance here.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              <<
>> Opinions?
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m :
>> >> current)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > currentStateAsMap.put(mapper.apply(m),
>> >> > > > > > >> m);
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > currentStateAsMap.put(toModifyKey,
>> >> > > > > > >> value);
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > currentStateAsMap.remove(toModifyKey);
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > if(currentStateAsMap.isEmpty()){
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return
>> >> null;
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
>> >> > > > > > >>>>>>> asAggregateType(currentStateAsMap)
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at
>> >> 3:35
>> >> > PM,
>> >> > > > Jan
>> >> > > > > > >>>>> Filipiak
>> >> > > > > > >>>>>>> <
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com
>> >> <mailto:
>> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for
>> bringing
>> >> > > Matthias
>> >> > > > > to
>> >> > > > > > >>>>> speed!
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the
>> differences. I
>> >> > think
>> >> > > > > > >> re-keying
>> >> > > > > > >>>>>>> back
>> >> > > > > > >>>>>>>>>>>> should be
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           optional at
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return
>> a
>> >> > > > > > KScatteredTable
>> >> > > > > > >>>>> with
>> >> > > > > > >>>>>>>>>>>> reshuffle()
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           returning
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> KTable<originalKey,Joined>
>> >> > to
>> >> > > > make
>> >> > > > > > the
>> >> > > > > > >>>>>>> backwards
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           repartitioning
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big
>> >> favour of
>> >> > > > doing
>> >> > > > > > the
>> >> > > > > > >>>>> out
>> >> > > > > > >>>>>>> of
>> >> > > > > > >>>>>>>>> order
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           processing using
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water
>> >> mark
>> >> > > > > tracking.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded
>> >> > growth
>> >> > > is
>> >> > > > > > just
>> >> > > > > > >>>>> scary
>> >> > > > > > >>>>>>> + It
>> >> > > > > > >>>>>>>>>>>> saves
>> >> > > > > > >>>>>>>>>>>>>>>>>>> us
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           the header
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the
>> abstraction
>> >> of
>> >> > > > always
>> >> > > > > > >>>>>>> repartitioning
>> >> > > > > > >>>>>>>>>>>> back is
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           just not so
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work
>> has
>> >> > been
>> >> > > > > done
>> >> > > > > > >>>>> before
>> >> > > > > > >>>>>>> we
>> >> > > > > > >>>>>>>>>>>> partition
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           back and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else
>> >> afterwards
>> >> > > is
>> >> > > > > > really
>> >> > > > > > >>>>>>> common.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49,
>> Adam
>> >> > > > > Bellemare
>> >> > > > > > >>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your
>> >> > feedback,
>> >> > > I
>> >> > > > do
>> >> > > > > > >>>>>>> appreciate
>> >> > > > > > >>>>>>>>> it!
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing
>> >> would be
>> >> > > > > > possible,
>> >> > > > > > >> it
>> >> > > > > > >>>>>>> would
>> >> > > > > > >>>>>>>>>>>> require
>> >> > > > > > >>>>>>>>>>>>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > deserialize
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
>> >> implies
>> >> > a
>> >> > > > > > runtime
>> >> > > > > > >>>>>>> overhead.
>> >> > > > > > >>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > no
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
>> >> avoid
>> >> > > the
>> >> > > > > > >>>>> overhead.
>> >> > > > > > >>>>>>> If
>> >> > > > > > >>>>>>>>> this
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> becomes a
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > problem in
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
>> >> still
>> >> > add
>> >> > > > > name
>> >> > > > > > >>>>> spacing
>> >> > > > > > >>>>>>>>> later
>> >> > > > > > >>>>>>>>>>>> on.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go
>> with
>> >> > > using a
>> >> > > > > > >> reserved
>> >> > > > > > >>>>>>> string
>> >> > > > > > >>>>>>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           document it.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about
>> >> the
>> >> > > > design
>> >> > > > > it
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> type of
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > If
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the
>> proposal
>> >> > > > > correctly,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you
>> >> have
>> >> > > > table1
>> >> > > > > > and
>> >> > > > > > >>>>> table2
>> >> > > > > > >>>>>>>>>>>> swapped.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           Here is how it
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the
>> >> records
>> >> > > that
>> >> > > > > > contain
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>>>>> foreign key
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           within their
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
>> >> > > > > > <a,(fk=A,bar=1)>,
>> >> > > > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream:
>> >> <A,X>,
>> >> > > > <B,Y>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is
>> >> > required
>> >> > > > to
>> >> > > > > > >> extract
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>> foreign
>> >> > > > > > >>>>>>>>>>>>>>>>>>> key.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key
>> >> mapper:
>> >> > (
>> >> > > > > value
>> >> > > > > > =>
>> >> > > > > > >>>>>>> value.fk
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is
>> applied to
>> >> > each
>> >> > > > > > element
>> >> > > > > > >> in
>> >> > > > > > >>>>>>>>> table1,
>> >> > > > > > >>>>>>>>>>>> and a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           new combined
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
>> >> > > > > (fk=A,bar=1)>,
>> >> > > > > > >>>>> <A-b,
>> >> > > > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           <B-c,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events
>> >> are
>> >> > > > > > >> copartitioned
>> >> > > > > > >>>>>>> with
>> >> > > > > > >>>>>>>>>>>> table2:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
>> >> > > Partition
>> >> > > > > 0:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
>> >> <A-a,
>> >> > > > > > >>>>> (fk=A,bar=1)>,
>> >> > > > > > >>>>>>> <A-b,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
>> >> > > Partition
>> >> > > > > 1:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
>> >> <B-c,
>> >> > > > > > >> (fk=B,bar=3)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they
>> can
>> >> be
>> >> > > > joined
>> >> > > > > > >>>>> together
>> >> > > > > > >>>>>>>>> locally
>> >> > > > > > >>>>>>>>>>>> by
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           applying the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's
>> >> design
>> >> > > and
>> >> > > > > my
>> >> > > > > > >>>>> design
>> >> > > > > > >>>>>>>>>>>> deviate. My
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           design goes
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > on
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
>> >> > > post-join
>> >> > > > > and
>> >> > > > > > >>>>> resolve
>> >> > > > > > >>>>>>>>>>>> out-of-order
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           arrival of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the
>> >> data
>> >> > > > keyed
>> >> > > > > > just
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>>>>>>> original key.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           I do not
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of
>> >> the
>> >> > > > > internals
>> >> > > > > > >>>>>>> outside of
>> >> > > > > > >>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does
>> make
>> >> > for
>> >> > > > > larger
>> >> > > > > > >>>>>>> footprint,
>> >> > > > > > >>>>>>>>>>>> but it
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           removes all
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving
>> >> out-of-order
>> >> > > > > arrivals
>> >> > > > > > >> and
>> >> > > > > > >>>>>>>>> handling
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that
>> >> this
>> >> > > makes
>> >> > > > > the
>> >> > > > > > >>>>>>> function
>> >> > > > > > >>>>>>>>> much
>> >> > > > > > >>>>>>>>>>>>>>>>>>> easier
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           to use.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this
>> >> helps
>> >> > > > resolve
>> >> > > > > > your
>> >> > > > > > >>>>>>>>> questions,
>> >> > > > > > >>>>>>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           please feel
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on
>> >> your
>> >> > > mind.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018
>> at
>> >> 8:36
>> >> > > PM,
>> >> > > > > > >>>>> Matthias J.
>> >> > > > > > >>>>>>>>> Sax <
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> matthias@confluent.io
>> >> > > <mailto:
>> >> > > > > > >>>>>>>>>>>> matthias@confluent.io>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching
>> up
>> >> on
>> >> > > this
>> >> > > > > > >> thread. I
>> >> > > > > > >>>>>>> did
>> >> > > > > > >>>>>>>>> not
>> >> > > > > > >>>>>>>>>>>> read
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           everything so
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share
>> >> couple
>> >> > of
>> >> > > > > > initial
>> >> > > > > > >>>>>>> thoughts:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think
>> there
>> >> is
>> >> > a
>> >> > > > > > >> fundamental
>> >> > > > > > >>>>>>>>>>>> difference
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           between header
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and
>> KP-258.
>> >> > For
>> >> > > > 258,
>> >> > > > > > we
>> >> > > > > > >> add
>> >> > > > > > >>>>>>>>> headers
>> >> > > > > > >>>>>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka
>> >> Streams
>> >> > > and
>> >> > > > > > nobody
>> >> > > > > > >>>>>>> else is
>> >> > > > > > >>>>>>>>>>>> supposed
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           to write
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > into
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no
>> user
>> >> > > header
>> >> > > > > are
>> >> > > > > > >>>>> written
>> >> > > > > > >>>>>>> into
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
>> >> > > conflicts.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I
>> don't
>> >> see
>> >> > a
>> >> > > > big
>> >> > > > > > issue
>> >> > > > > > >>>>> with
>> >> > > > > > >>>>>>>>> using
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           headers within
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we
>> document
>> >> it,
>> >> > > we
>> >> > > > > can
>> >> > > > > > >> have
>> >> > > > > > >>>>>>> some
>> >> > > > > > >>>>>>>>>>>> "reserved"
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           header keys
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not
>> allowed
>> >> to
>> >> > use
>> >> > > > > when
>> >> > > > > > >>>>>>> processing
>> >> > > > > > >>>>>>>>>>>> data with
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           Kafka
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Streams.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be
>> >> ok.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a
>> safe
>> >> > way
>> >> > > to
>> >> > > > > > avoid
>> >> > > > > > >>>>>>>>> conflicts,
>> >> > > > > > >>>>>>>>>>>> since
>> >> > > > > > >>>>>>>>>>>>>>>>>>> these
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > headers
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in
>> >> internal
>> >> > > > topics
>> >> > > > > (I
>> >> > > > > > >>>>> think):
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and
>> >> > changelog
>> >> > > > > > topics,
>> >> > > > > > >> we
>> >> > > > > > >>>>> can
>> >> > > > > > >>>>>>>>>>>> namespace
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           all headers:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined
>> headers
>> >> > are
>> >> > > > > > >> namespaced
>> >> > > > > > >>>>> as
>> >> > > > > > >>>>>>>>>>>> "external."
>> >> > > > > > >>>>>>>>>>>>>>>>>>> +
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers
>> are
>> >> > > > > > namespaced as
>> >> > > > > > >>>>>>>>>>>> "internal." +
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing
>> >> would
>> >> > be
>> >> > > > > > >> possible,
>> >> > > > > > >>>>> it
>> >> > > > > > >>>>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>>>> require
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
>> >> implies
>> >> > a
>> >> > > > > > runtime
>> >> > > > > > >>>>>>> overhead.
>> >> > > > > > >>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > no
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
>> >> avoid
>> >> > > the
>> >> > > > > > >>>>> overhead.
>> >> > > > > > >>>>>>> If
>> >> > > > > > >>>>>>>>> this
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> becomes a
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > problem in
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
>> >> still
>> >> > add
>> >> > > > > name
>> >> > > > > > >>>>> spacing
>> >> > > > > > >>>>>>>>> later
>> >> > > > > > >>>>>>>>>>>> on.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern
>> about
>> >> the
>> >> > > > design
>> >> > > > > > it
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> type
>> >> > > > > > >>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the
>> proposal
>> >> > > > > correctly,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1>
>> table1 =
>> >> ...
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2>
>> table2 =
>> >> ...
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3>
>> >> joinedTable
>> >> > =
>> >> > > > > > >>>>>>>>>>>> table1.join(table2,...);
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
>> >> > > `joinedTable`
>> >> > > > > has
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> same
>> >> > > > > > >>>>>>>>> key
>> >> > > > > > >>>>>>>>>>>> as the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           left input
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not
>> >> work
>> >> > > > because
>> >> > > > > > if
>> >> > > > > > >>>>> table2
>> >> > > > > > >>>>>>>>>>>> contains
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           multiple rows
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record
>> in
>> >> > table1
>> >> > > > > > (what is
>> >> > > > > > >>>>> the
>> >> > > > > > >>>>>>> main
>> >> > > > > > >>>>>>>>>>>> purpose
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > foreign
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result
>> table
>> >> > > would
>> >> > > > > only
>> >> > > > > > >>>>>>> contain a
>> >> > > > > > >>>>>>>>>>>> single
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           join result,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > but
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream:
>> >> <A,X>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
>> >> > > > <a,(A,1)>,
>> >> > > > > > >>>>> <b,(A,2)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value
>> a
>> >> > > foreign
>> >> > > > > key
>> >> > > > > > to
>> >> > > > > > >>>>>>> table1
>> >> > > > > > >>>>>>>>> key
>> >> > > > > > >>>>>>>>>>>> (ie,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > If
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the
>> same
>> >> key
>> >> > > as
>> >> > > > > key
>> >> > > > > > of
>> >> > > > > > >>>>>>> table1,
>> >> > > > > > >>>>>>>>> this
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           implies that the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be
>> >> <A,
>> >> > > > > > join(X,1)>
>> >> > > > > > >> or
>> >> > > > > > >>>>> <A,
>> >> > > > > > >>>>>>>>>>>> join(X,2)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           but not
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > both.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share
>> the
>> >> same
>> >> > > > key,
>> >> > > > > > >>>>> whatever
>> >> > > > > > >>>>>>>>> result
>> >> > > > > > >>>>>>>>>>>> record
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           we emit
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > later,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the
>> previous
>> >> > > result.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason
>> why
>> >> Jan
>> >> > > > > > originally
>> >> > > > > > >>>>>>> proposed
>> >> > > > > > >>>>>>>>>>>> to use
>> >> > > > > > >>>>>>>>>>>>>>>>>>> a
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > combination
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of
>> >> the
>> >> > > input
>> >> > > > > > tables
>> >> > > > > > >>>>> as
>> >> > > > > > >>>>>>> key
>> >> > > > > > >>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           output table.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of
>> the
>> >> > output
>> >> > > > > table
>> >> > > > > > >>>>> unique
>> >> > > > > > >>>>>>> and
>> >> > > > > > >>>>>>>>> we
>> >> > > > > > >>>>>>>>>>>> can
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           store both in
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be
>> <A-a,
>> >> > > > > join(X,1)>,
>> >> > > > > > >> <A-b,
>> >> > > > > > >>>>>>>>>>>> join(X,2)>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM,
>> Jan
>> >> > > > Filipiak
>> >> > > > > > >> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark
>> >> could
>> >> > be
>> >> > > > > > >>>>> disregarded.
>> >> > > > > > >>>>>>> The
>> >> > > > > > >>>>>>>>>>>> decision
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           about the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the
>> size of
>> >> > the
>> >> > > > > > >> aggregated
>> >> > > > > > >>>>>>> map.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long
>> >> maps
>> >> > > > would
>> >> > > > > be
>> >> > > > > > >>>>>>> unpacked
>> >> > > > > > >>>>>>>>> and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > element
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published
>> as
>> >> > > delete.
>> >> > > > > Any
>> >> > > > > > >>>>> other
>> >> > > > > > >>>>>>> count
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is
>> in
>> >> > > "waiting
>> >> > > > > for
>> >> > > > > > >>>>> correct
>> >> > > > > > >>>>>>>>>>>> deletes to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018
>> 21:29,
>> >> Adam
>> >> > > > > > Bellemare
>> >> > > > > > >>>>>>> wrote:
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I
>> >> could
>> >> > > > > replace
>> >> > > > > > >> the
>> >> > > > > > >>>>>>> second
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           repartition store
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > and
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store
>> with
>> >> a
>> >> > > > groupBy
>> >> > > > > > and
>> >> > > > > > >>>>>>> reduce.
>> >> > > > > > >>>>>>>>>>>> However,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           it looks
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > like
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to
>> store
>> >> the
>> >> > > > > > highwater
>> >> > > > > > >>>>> value
>> >> > > > > > >>>>>>>>> within
>> >> > > > > > >>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           materialized
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the
>> arrival of
>> >> > > > > > out-of-order
>> >> > > > > > >>>>>>> records
>> >> > > > > > >>>>>>>>>>>> (assuming
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>> my
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is
>> correct...).
>> >> This
>> >> > > in
>> >> > > > > > effect
>> >> > > > > > >> is
>> >> > > > > > >>>>>>> the
>> >> > > > > > >>>>>>>>> same
>> >> > > > > > >>>>>>>>>>>> as
>> >> > > > > > >>>>>>>>>>>>>>>>>>> the
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           design I
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two
>> >> tables
>> >> > > > merged
>> >> > > > > > >>>>> together.
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>> --
>> >> > > > > > >>>>>>>>>>>>>>>>> -- Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>> --
>> >> > > > > > >>>>>>>>>>>>>>>> -- Guozhang
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>>
>> >> > > > > > >>>>>>>>>>
>> >> > > > > > >>>>>>>>>
>> >> > > > > > >>>>>>>>
>> >> > > > > > >>>>>>>
>> >> > > > > > >>>>>>
>> >> > > > > > >>>>>
>> >> > > > > > >>>>
>> >> > > > > > >>>
>> >> > > > > > >>
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >> >
>> >> > --
>> >> > -- Guozhang
>> >> >
>> >>
>> >
>>
>> --
>> -- Guozhang
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi John and Guozhang

Ah yes, I lost that in the mix! Thanks for the convergent solutions - I do
think that the attachment that John included makes for a better design. It
should also help with overall performance as very high-cardinality foreign
keyed data (say millions of events with the same entity) will be able to
leverage the multiple nodes for join functionality instead of having it all
performed in one node. There is still a bottleneck in the right table
having to propagate all those events, but with slimmer structures, less IO
and no need to perform the join I think the throughput will be much higher
in those scenarios.

Okay, I am convinced. I will update the KIP accordingly to a Gliffy version
of John's diagram and ensure that the example flow matches correctly. Then
I can go back to working on the PR to match the diagram.

Thanks both of you for all the help - very much appreciated.

Adam







On Mon, Dec 17, 2018 at 6:39 PM Guozhang Wang <wa...@gmail.com> wrote:

> Hi John,
>
> Just made a pass on your diagram (nice hand-drawing btw!), and obviously we
> are thinking about the same thing :) A neat difference that I like, is that
> in the pre-join repartition topic we can still send message in the format
> of `K=k, V=(i=2)` while using "i" as the partition key in StreamsPartition,
> this way we do not need to even augment the key for the repartition topic,
> but just do a projection on the foreign key part but trim all other fields:
> as long as we still materialize the store as `A-2` co-located with the
> right KTable, that is fine.
>
> As I mentioned in my previous email, I also think this has a few advantages
> on saving over-the-wire bytes as well as disk bytes.
>
> Guozhang
>
>
> On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io> wrote:
>
> > Hi Guozhang,
> >
> > Thanks for taking a look! I think Adam's already addressed your questions
> > as well as I could have.
> >
> > Hi Adam,
> >
> > Thanks for updating the KIP. It looks great, especially how all the
> > need-to-know information is right at the top, followed by the details.
> >
> > Also, thanks for that high-level diagram. Actually, now that I'm looking
> > at it, I think part of my proposal got lost in translation, although I do
> > think that what you have there is also correct.
> >
> > I sketched up a crude diagram based on yours and attached it to the KIP
> > (I'm not sure if attached or inline images work on the mailing list):
> >
> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
> > . It's also attached to this email for convenience.
> >
> > Hopefully, you can see how it's intended to line up, and which parts are
> > modified.
> > At a high level, instead of performing the join on the right-hand side,
> > we're essentially just registering interest, like "LHS key A wishes to
> > receive updates for RHS key 2". Then, when there is a new "interest" or
> any
> > updates to the RHS records, it "broadcasts" its state back to the LHS
> > records who are interested in it.
> >
> > Thus, instead of sending the LHS values to the RHS joiner workers and
> then
> > sending the join results back to the LHS worke be co-partitioned and
> > validated, we instead only send the LHS *keys* to the RHS workers and
> then
> > only the RHS k/v back to be joined by the LHS worker.
> >
> > I've been considering both your diagram and mine, and I *think* what I'm
> > proposing has a few advantages.
> >
> > Here are some points of interest as you look at the diagram:
> > * When we extract the foreign key and send it to the Pre-Join Repartition
> > Topic, we can send only the FK/PK pair. There's no need to worry about
> > custom partitioner logic, since we can just use the foreign key plainly
> as
> > the repartition record key. Also, we save on transmitting the LHS value,
> > since we only send its key in this step.
> > * We also only need to store the RHSKey:LHSKey mapping in the
> > MaterializedSubscriptionStore, saving on disk. We can use the same rocks
> > key format you proposed and the same algorithm involving range scans when
> > the RHS records get updated.
> > * Instead of joining on the right side, all we do is compose a
> > re-repartition record so we can broadcast the RHS k/v pair back to the
> > original LHS partition. (this is what the "rekey" node is doing)
> > * Then, there is a special kind of Joiner that's co-resident in the same
> > StreamTask as the LHS table, subscribed to the Post-Join Repartition
> Topic.
> > ** This Joiner is *not* triggered directly by any changes in the LHS
> > KTable. Instead, LHS events indirectly trigger the join via the whole
> > lifecycle.
> > ** For each event arriving from the Post-Join Repartition Topic, the
> > Joiner looks up the corresponding record in the LHS KTable. It validates
> > the FK as you noted, discarding any inconsistent events. Otherwise, it
> > unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the join
> > result
> > ** Note that the Joiner itself is stateless, so materializing the join
> > result is optional, just as with the 1:1 joins.
> >
> > So in summary:
> > * instead of transmitting the LHS keys and values to the right and the
> > JoinResult back to the left, we only transmit the LHS keys to the right
> and
> > the RHS values to the left. Assuming the average RHS value is on smaller
> > than or equal to the average join result size, it's a clear win on broker
> > traffic. I think this is actually a reasonable assumption, which we can
> > discuss more if you're suspicious.
> > * we only need one copy of the data (the left and right tables need to be
> > materialized) and one extra copy of the PK:FK pairs in the Materialized
> > Subscription Store. Materializing the join result is optional, just as
> with
> > the existing 1:1 joins.
> > * we still need the fancy range-scan algorithm on the right to locate all
> > interested LHS keys when a RHS value is updated, but we don't need a
> custom
> > partitioner for either repartition topic (this is of course a
> modification
> > we could make to your version as well)
> >
> > How does this sound to you? (And did I miss anything?)
> > -John
> >
> > On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <adam.bellemare@gmail.com
> >
> > wrote:
> >
> >> Hi John & Guozhang
> >>
> >> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the
> KIP,
> >> pruned much of what I wrote and put a simplified diagram near the top to
> >> illustrate the workflow. I encapsulated Jan's content at the bottom of
> the
> >> document. I believe it is simpler to read by far now.
> >>
> >> @Guozhang Wang <wa...@gmail.com>:
> >> > #1: rekey left table
> >> >   -> source from the left upstream, send to rekey-processor to
> generate
> >> combined key, and then sink to copartition topic.
> >> Correct.
> >>
> >> > #2: first-join with right table
> >> >   -> source from the right table upstream, materialize the right
> table.
> >> >   -> source from the co-partition topic, materialize the rekeyed left
> >> table, join with the right table, rekey back, and then sink to the
> >> rekeyed-back topic.
> >> Almost - I cleared up the KIP. We do not rekey back yet, as I need the
> >> Foreign-Key value generated in #1 above to compare in the resolution
> >> stage.
> >>
> >> > #3: second join
> >> >    -> source from the rekeyed-back topic, materialize the rekeyed back
> >> table.
> >> >   -> source from the left upstream, materialize the left table, join
> >> with
> >> the rekeyed back table.
> >> Almost - As each event comes in, we just run it through a stateful
> >> processor that checks the original ("This") KTable for the key. The
> value
> >> payload then has the foreignKeyExtractor applied again as in Part #1
> >> above,
> >> and gets the current foreign key. Then we compare it to the joined event
> >> that we are currently resolving. If they have the same foreign-key,
> >> propagate the result out. If they don't, throw the event away.
> >>
> >> The end result is that we do need to materialize 2 additional tables
> >> (left/this-combinedkey table, and the final Joined table) as I've
> >> illustrated in the updated KIP. I hope the diagram clears it up a lot
> >> better. Please let me know.
> >>
> >> Thanks again
> >> Adam
> >>
> >>
> >>
> >>
> >> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >> > John,
> >> >
> >> > Thanks a lot for the suggestions on refactoring the wiki, I agree with
> >> you
> >> > that we should consider the KIP proposal to be easily understood by
> >> anyone
> >> > in the future to read, and hence should provide a good summary on the
> >> > user-facing interfaces, as well as rejected alternatives to represent
> >> > briefly "how we came a long way to this conclusion, and what we have
> >> > argued, disagreed, and agreed about, etc" so that readers do not need
> to
> >> > dig into the DISCUSS thread to get all the details. We can, of course,
> >> keep
> >> > the implementation details like "workflows" on the wiki page as a
> >> addendum
> >> > section since it also has correlations.
> >> >
> >> > Regarding your proposal on comment 6): that's a very interesting idea!
> >> Just
> >> > to clarify that I understands it fully correctly: the proposal's
> >> resulted
> >> > topology is still the same as the current proposal, where we will
> have 3
> >> > sub-topologies for this operator:
> >> >
> >> > #1: rekey left table
> >> >    -> source from the left upstream, send to rekey-processor to
> generate
> >> > combined key, and then sink to copartition topic.
> >> >
> >> > #2: first-join with right table
> >> >    -> source from the right table upstream, materialize the right
> table.
> >> >    -> source from the co-partition topic, materialize the rekeyed left
> >> > table, join with the right table, rekey back, and then sink to the
> >> > rekeyed-back topic.
> >> >
> >> > #3: second join
> >> >    -> source from the rekeyed-back topic, materialize the rekeyed back
> >> > table.
> >> >    -> source from the left upstream, materialize the left table, join
> >> with
> >> > the rekeyed back table.
> >> >
> >> > Sub-topology #1 and #3 may be merged to a single sub-topology since
> >> both of
> >> > them read from the left table source stream. In this workflow, we need
> >> to
> >> > materialize 4 tables (left table in #3, right table in #2, rekeyed
> left
> >> > table in #2, rekeyed-back table in #3), and 2 repartition topics
> >> > (copartition topic, rekeyed-back topic).
> >> >
> >> > Compared with Adam's current proposal in the workflow overview, it has
> >> the
> >> > same num.materialize tables (left table, rekeyed left table, right
> >> table,
> >> > out-of-ordering resolver table), and same num.internal topics (two).
> The
> >> > advantage is that on the copartition topic, we can save bandwidth by
> not
> >> > sending value, and in #2 the rekeyed left table is smaller since we do
> >> not
> >> > have any values to materialize. Is that right?
> >> >
> >> >
> >> > Guozhang
> >> >
> >> >
> >> >
> >> > On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io>
> wrote:
> >> >
> >> > > Hi Adam,
> >> > >
> >> > > Given that the committers are all pretty busy right now, I think
> that
> >> it
> >> > > would help if you were to refactor the KIP a little to reduce the
> >> > workload
> >> > > for reviewers.
> >> > >
> >> > > I'd recommend the following changes:
> >> > > * relocate all internal details to a section at the end called
> >> something
> >> > > like "Implementation Notes" or something like that.
> >> > > * rewrite the rest of the KIP to be a succinct as possible and
> mention
> >> > only
> >> > > publicly-facing API changes.
> >> > > ** for example, the interface that you've already listed there, as
> >> well
> >> > as
> >> > > a textual description of the guarantees we'll be providing (join
> >> result
> >> > is
> >> > > copartitioned with the LHS, and the join result is guaranteed
> correct)
> >> > >
> >> > > A good target would be that the whole main body of the KIP,
> including
> >> > > Status, Motivation, Proposal, Justification, and Rejected
> Alternatives
> >> > all
> >> > > fit "above the fold" (i.e., all fit on the screen at a comfortable
> >> zoom
> >> > > level).
> >> > > I think the only real Rejected Alternative that bears mention at
> this
> >> > point
> >> > > is KScatteredTable, which you could just include the executive
> >> summary on
> >> > > (no implementation details), and link to extra details in the
> >> > > Implementation Notes section.
> >> > >
> >> > > Taking a look at the wiki page, ~90% of the text there is internal
> >> > detail,
> >> > > which is useful for the dubious, but doesn't need to be ratified in
> a
> >> > vote
> >> > > (and would be subject to change without notice in the future
> anyway).
> >> > > There's also a lot of conflicting discussion, as you've very
> >> respectfully
> >> > > tried to preserve the original proposal from Jan while adding your
> >> own.
> >> > > Isolating all this information in a dedicated section at the bottom
> >> frees
> >> > > the voters up to focus on the public API part of the proposal, which
> >> is
> >> > > really all they need to consider.
> >> > >
> >> > > Plus, it'll be clear to future readers which parts of the document
> are
> >> > > enduring, and which parts are a snapshot of our implementation
> >> thinking
> >> > at
> >> > > the time.
> >> > >
> >> > > I'm suggesting this because I suspect that the others haven't made
> >> time
> >> > to
> >> > > review it partly because it seems daunting. If it seems like it
> would
> >> be
> >> > a
> >> > > huge time investment to review, people will just keep putting it
> off.
> >> But
> >> > > if the KIP is a single page, then they'll be more inclined to give
> it
> >> a
> >> > > read.
> >> > >
> >> > > Honestly, I don't think the KIP itself is that controversial (apart
> >> from
> >> > > the scattered table thing (sorry, Jan) ). Most of the discussion has
> >> been
> >> > > around the implementation, which we can continue more effectively in
> >> a PR
> >> > > once the KIP has passed.
> >> > >
> >> > > How does that sound?
> >> > > -John
> >> > >
> >> > > On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
> >> adam.bellemare@gmail.com
> >> > >
> >> > > wrote:
> >> > >
> >> > > > 1) I believe that the resolution mechanism John has proposed is
> >> > > sufficient
> >> > > > - it is clean and easy and doesn't require additional RocksDB
> >> stores,
> >> > > which
> >> > > > reduces the footprint greatly. I don't think we need to resolve
> >> based
> >> > on
> >> > > > timestamp or offset anymore, but if we decide to do to that would
> be
> >> > > within
> >> > > > the bounds of the existing API.
> >> > > >
> >> > > > 2) Is the current API sufficient, or does it need to be altered to
> >> go
> >> > > back
> >> > > > to vote?
> >> > > >
> >> > > > 3) KScatteredTable implementation can always be added in a future
> >> > > revision.
> >> > > > This API does not rule it out. This implementation of this
> function
> >> > would
> >> > > > simply be replaced with `KScatteredTable.resolve()` while still
> >> > > maintaining
> >> > > > the existing API, thereby giving both features as Jan outlined
> >> earlier.
> >> > > > Would this work?
> >> > > >
> >> > > >
> >> > > > Thanks Guozhang, John and Jan
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io>
> >> > wrote:
> >> > > >
> >> > > > > Hi, all,
> >> > > > >
> >> > > > > >> In fact, we
> >> > > > > >> can just keep a single final-result store with timestamps and
> >> > reject
> >> > > > > values
> >> > > > > >> that have a smaller timestamp, is that right?
> >> > > > >
> >> > > > > > Which is the correct output should at least be decided on the
> >> > offset
> >> > > of
> >> > > > > > the original message.
> >> > > > >
> >> > > > > Thanks for this point, Jan.
> >> > > > >
> >> > > > > KIP-258 is merely to allow embedding the record timestamp  in
> the
> >> k/v
> >> > > > > store,
> >> > > > > as well as providing a storage-format upgrade path.
> >> > > > >
> >> > > > > I might have missed it, but I think we have yet to discuss
> whether
> >> > it's
> >> > > > > safe
> >> > > > > or desirable just to swap topic-ordering our for
> >> timestamp-ordering.
> >> > > This
> >> > > > > is
> >> > > > > a very deep topic, and I think it would only pollute the current
> >> > > > > discussion.
> >> > > > >
> >> > > > > What Adam has proposed is safe, given the *current* ordering
> >> > semantics
> >> > > > > of the system. If we can agree on his proposal, I think we can
> >> merge
> >> > > the
> >> > > > > feature well before the conversation about timestamp ordering
> even
> >> > > takes
> >> > > > > place, much less reaches a conclusion. In the mean time, it
> would
> >> > seem
> >> > > to
> >> > > > > be unfortunate to have one join operator with different ordering
> >> > > > semantics
> >> > > > > from every other KTable operator.
> >> > > > >
> >> > > > > If and when that timestamp discussion takes place, many (all?)
> >> KTable
> >> > > > > operations
> >> > > > > will need to be updated, rendering the many:one join a small
> >> marginal
> >> > > > cost.
> >> > > > >
> >> > > > > And, just to plug it again, I proposed an algorithm above that I
> >> > > believe
> >> > > > > provides
> >> > > > > correct ordering without any additional metadata, and regardless
> >> of
> >> > the
> >> > > > > ordering semantics. I didn't bring it up further, because I felt
> >> the
> >> > > KIP
> >> > > > > only needs
> >> > > > > to agree on the public API, and we can discuss the
> implementation
> >> at
> >> > > > > leisure in
> >> > > > > a PR...
> >> > > > >
> >> > > > > Thanks,
> >> > > > > -John
> >> > > > >
> >> > > > >
> >> > > > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> >> > Jan.Filipiak@trivago.com
> >> > > >
> >> > > > > wrote:
> >> > > > >
> >> > > > > >
> >> > > > > >
> >> > > > > > On 10.12.2018 07:42, Guozhang Wang wrote:
> >> > > > > > > Hello Adam / Jan / John,
> >> > > > > > >
> >> > > > > > > Sorry for being late on this thread! I've finally got some
> >> time
> >> > > this
> >> > > > > > > weekend to cleanup a load of tasks on my queue (actually
> I've
> >> > also
> >> > > > > > realized
> >> > > > > > > there are a bunch of other things I need to enqueue while
> >> > cleaning
> >> > > > them
> >> > > > > > up
> >> > > > > > > --- sth I need to improve on my side). So here are my
> >> thoughts:
> >> > > > > > >
> >> > > > > > > Regarding the APIs: I like the current written API in the
> KIP.
> >> > More
> >> > > > > > > generally I'd prefer to keep the 1) one-to-many join
> >> > > functionalities
> >> > > > as
> >> > > > > > > well as 2) other join types than inner as separate KIPs
> since
> >> 1)
> >> > > may
> >> > > > > > worth
> >> > > > > > > a general API refactoring that can benefit not only
> foreignkey
> >> > > joins
> >> > > > > but
> >> > > > > > > collocate joins as well (e.g. an extended proposal of
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> >> > > > > > ),
> >> > > > > > > and I'm not sure if other join types would actually be
> needed
> >> > > (maybe
> >> > > > > left
> >> > > > > > > join still makes sense), so it's better to
> >> > > > > wait-for-people-to-ask-and-add
> >> > > > > > > than add-sth-that-no-one-uses.
> >> > > > > > >
> >> > > > > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> >> > > > > > > KScatteredTable for users to inject their own optimization:
> >> I'd
> >> > > > prefer
> >> > > > > to
> >> > > > > > > do the current option as-is, and my main rationale is for
> >> > > > optimization
> >> > > > > > > rooms inside the Streams internals and the API succinctness.
> >> For
> >> > > > > advanced
> >> > > > > > > users who may indeed prefer KScatteredTable and do their own
> >> > > > > > optimization,
> >> > > > > > > while it is too much of the work to use Processor API
> >> directly, I
> >> > > > think
> >> > > > > > we
> >> > > > > > > can still extend the current API to support it in the future
> >> if
> >> > it
> >> > > > > > becomes
> >> > > > > > > necessary.
> >> > > > > >
> >> > > > > > no internal optimization potential. it's a myth
> >> > > > > >
> >> > > > > > ¯\_(ツ)_/¯
> >> > > > > >
> >> > > > > > :-)
> >> > > > > >
> >> > > > > > >
> >> > > > > > > Another note about step 4) resolving out-of-ordering data,
> as
> >> I
> >> > > > > mentioned
> >> > > > > > > before I think with KIP-258 (embedded timestamp with
> key-value
> >> > > store)
> >> > > > > we
> >> > > > > > > can actually make this step simpler than the current
> >> proposal. In
> >> > > > fact,
> >> > > > > > we
> >> > > > > > > can just keep a single final-result store with timestamps
> and
> >> > > reject
> >> > > > > > values
> >> > > > > > > that have a smaller timestamp, is that right?
> >> > > > > >
> >> > > > > > Which is the correct output should at least be decided on the
> >> > offset
> >> > > of
> >> > > > > > the original message.
> >> > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > That's all I have in mind now. Again, great appreciation to
> >> Adam
> >> > to
> >> > > > > make
> >> > > > > > > such HUGE progress on this KIP!
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > Guozhang
> >> > > > > > >
> >> > > > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> >> > > > Jan.Filipiak@trivago.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > >> If they don't find the time:
> >> > > > > > >> They usually take the opposite path from me :D
> >> > > > > > >> so the answer would be clear.
> >> > > > > > >>
> >> > > > > > >> hence my suggestion to vote.
> >> > > > > > >>
> >> > > > > > >>
> >> > > > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> >> > > > > > >>> Hi Guozhang and Matthias
> >> > > > > > >>>
> >> > > > > > >>> I know both of you are quite busy, but we've gotten this
> KIP
> >> > to a
> >> > > > > point
> >> > > > > > >>> where we need more guidance on the API (perhaps a bit of a
> >> > > > > tie-breaker,
> >> > > > > > >> if
> >> > > > > > >>> you will). If you have anyone else you may think should
> >> look at
> >> > > > this,
> >> > > > > > >>> please tag them accordingly.
> >> > > > > > >>>
> >> > > > > > >>> The scenario is as such:
> >> > > > > > >>>
> >> > > > > > >>> Current Option:
> >> > > > > > >>> API:
> >> > > > > > >>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> >> > > > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
> >> > > partition
> >> > > > > with
> >> > > > > > >> the
> >> > > > > > >>> foreignKey (repartition 1)
> >> > > > > > >>> 2) Join the data
> >> > > > > > >>> 3) Shuffle the data back to the original node (repartition
> >> 2)
> >> > > > > > >>> 4) Resolve out-of-order arrival / race condition due to
> >> > > foreign-key
> >> > > > > > >> changes.
> >> > > > > > >>>
> >> > > > > > >>> Alternate Option:
> >> > > > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
> >> > > > > > >>> - It would be keyed on a wrapped key function:
> >> <CombinedKey<KO,
> >> > > K>,
> >> > > > > VR>
> >> > > > > > >> (KO
> >> > > > > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> >> > > > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
> >> > > otherwise a
> >> > > > > > user
> >> > > > > > >>> would be able to perform additional functions directly
> from
> >> the
> >> > > > > > >>> KScatteredTable (TBD - currently out of scope).
> >> > > > > > >>> - John's analysis 2-emails up is accurate as to the
> >> tradeoffs.
> >> > > > > > >>>
> >> > > > > > >>> Current Option is coded as-is. Alternate option is
> possible,
> >> > but
> >> > > > will
> >> > > > > > >>> require for implementation details to be made in the API
> and
> >> > some
> >> > > > > > >> exposure
> >> > > > > > >>> of new data structures into the API (ie: CombinedKey).
> >> > > > > > >>>
> >> > > > > > >>> I appreciate any insight into this.
> >> > > > > > >>>
> >> > > > > > >>> Thanks.
> >> > > > > > >>>
> >> > > > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> >> > > > > > adam.bellemare@gmail.com>
> >> > > > > > >>> wrote:
> >> > > > > > >>>
> >> > > > > > >>>> Hi John
> >> > > > > > >>>>
> >> > > > > > >>>> Thanks for your feedback and assistance. I think your
> >> summary
> >> > is
> >> > > > > > >> accurate
> >> > > > > > >>>> from my perspective. Additionally, I would like to add
> that
> >> > > there
> >> > > > > is a
> >> > > > > > >> risk
> >> > > > > > >>>> of inconsistent final states without performing the
> >> > resolution.
> >> > > > This
> >> > > > > > is
> >> > > > > > >> a
> >> > > > > > >>>> major concern for me as most of the data I have dealt
> with
> >> is
> >> > > > > produced
> >> > > > > > >> by
> >> > > > > > >>>> relational databases. We have seen a number of cases
> where
> >> a
> >> > > user
> >> > > > in
> >> > > > > > the
> >> > > > > > >>>> Rails UI has modified the field (foreign key), realized
> >> they
> >> > > made
> >> > > > a
> >> > > > > > >>>> mistake, and then updated the field again with a new key.
> >> The
> >> > > > events
> >> > > > > > are
> >> > > > > > >>>> propagated out as they are produced, and as such we have
> >> had
> >> > > > > > real-world
> >> > > > > > >>>> cases where these inconsistencies were propagated
> >> downstream
> >> > as
> >> > > > the
> >> > > > > > >> final
> >> > > > > > >>>> values due to the race conditions in the fanout of the
> >> data.
> >> > > > > > >>>>
> >> > > > > > >>>> This solution that I propose values correctness of the
> >> final
> >> > > > result
> >> > > > > > over
> >> > > > > > >>>> other factors.
> >> > > > > > >>>>
> >> > > > > > >>>> We could always move this function over to using a
> >> > > KScatteredTable
> >> > > > > > >>>> implementation in the future, and simply deprecate it
> this
> >> > join
> >> > > > API
> >> > > > > in
> >> > > > > > >>>> time. I think I would like to hear more from some of the
> >> other
> >> > > > major
> >> > > > > > >>>> committers on which course of action they would think is
> >> best
> >> > > > before
> >> > > > > > any
> >> > > > > > >>>> more coding is done.
> >> > > > > > >>>>
> >> > > > > > >>>> Thanks again
> >> > > > > > >>>>
> >> > > > > > >>>> Adam
> >> > > > > > >>>>
> >> > > > > > >>>>
> >> > > > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> >> > john@confluent.io>
> >> > > > > > wrote:
> >> > > > > > >>>>
> >> > > > > > >>>>> Hi Jan and Adam,
> >> > > > > > >>>>>
> >> > > > > > >>>>> Wow, thanks for doing that test, Adam. Those results are
> >> > > > > encouraging.
> >> > > > > > >>>>>
> >> > > > > > >>>>> Thanks for your performance experience as well, Jan. I
> >> agree
> >> > > that
> >> > > > > > >> avoiding
> >> > > > > > >>>>> unnecessary join outputs is especially important when
> the
> >> > > fan-out
> >> > > > > is
> >> > > > > > so
> >> > > > > > >>>>> high. I suppose this could also be built into the
> >> > > implementation
> >> > > > > > we're
> >> > > > > > >>>>> discussing, but it wouldn't have to be specified in the
> >> KIP
> >> > > > (since
> >> > > > > > >> it's an
> >> > > > > > >>>>> API-transparent optimization).
> >> > > > > > >>>>>
> >> > > > > > >>>>> As far as whether or not to re-repartition the data, I
> >> didn't
> >> > > > bring
> >> > > > > > it
> >> > > > > > >> up
> >> > > > > > >>>>> because it sounded like the two of you agreed to leave
> the
> >> > KIP
> >> > > > > as-is,
> >> > > > > > >>>>> despite the disagreement.
> >> > > > > > >>>>>
> >> > > > > > >>>>> If you want my opinion, I feel like both approaches are
> >> > > > reasonable.
> >> > > > > > >>>>> It sounds like Jan values more the potential for
> >> developers
> >> > to
> >> > > > > > optimize
> >> > > > > > >>>>> their topologies to re-use the intermediate nodes,
> whereas
> >> > Adam
> >> > > > > > places
> >> > > > > > >>>>> more
> >> > > > > > >>>>> value on having a single operator that people can use
> >> without
> >> > > > extra
> >> > > > > > >> steps
> >> > > > > > >>>>> at the end.
> >> > > > > > >>>>>
> >> > > > > > >>>>> Personally, although I do find it exceptionally annoying
> >> > when a
> >> > > > > > >> framework
> >> > > > > > >>>>> gets in my way when I'm trying to optimize something, it
> >> > seems
> >> > > > > better
> >> > > > > > >> to
> >> > > > > > >>>>> go
> >> > > > > > >>>>> for a single operation.
> >> > > > > > >>>>> * Encapsulating the internal transitions gives us
> >> significant
> >> > > > > > latitude
> >> > > > > > >> in
> >> > > > > > >>>>> the implementation (for example, joining only at the
> end,
> >> not
> >> > > in
> >> > > > > the
> >> > > > > > >>>>> middle
> >> > > > > > >>>>> to avoid extra data copying and out-of-order resolution;
> >> how
> >> > we
> >> > > > > > >> represent
> >> > > > > > >>>>> the first repartition keys (combined keys vs. value
> >> vectors),
> >> > > > > etc.).
> >> > > > > > >> If we
> >> > > > > > >>>>> publish something like a KScatteredTable with the
> >> > > > right-partitioned
> >> > > > > > >> joined
> >> > > > > > >>>>> data, then the API pretty much locks in the
> >> implementation as
> >> > > > well.
> >> > > > > > >>>>> * The API seems simpler to understand and use. I do mean
> >> > > "seems";
> >> > > > > if
> >> > > > > > >>>>> anyone
> >> > > > > > >>>>> wants to make the case that KScatteredTable is actually
> >> > > simpler,
> >> > > > I
> >> > > > > > >> think
> >> > > > > > >>>>> hypothetical usage code would help. From a relational
> >> algebra
> >> > > > > > >> perspective,
> >> > > > > > >>>>> it seems like KTable.join(KTable) should produce a new
> >> KTable
> >> > > in
> >> > > > > all
> >> > > > > > >>>>> cases.
> >> > > > > > >>>>> * That said, there might still be room in the API for a
> >> > > different
> >> > > > > > >>>>> operation
> >> > > > > > >>>>> like what Jan has proposed to scatter a KTable, and then
> >> do
> >> > > > things
> >> > > > > > like
> >> > > > > > >>>>> join, re-group, etc from there... I'm not sure; I
> haven't
> >> > > thought
> >> > > > > > >> through
> >> > > > > > >>>>> all the consequences yet.
> >> > > > > > >>>>>
> >> > > > > > >>>>> This is all just my opinion after thinking over the
> >> > discussion
> >> > > so
> >> > > > > > >> far...
> >> > > > > > >>>>> -John
> >> > > > > > >>>>>
> >> > > > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> >> > > > > > >> adam.bellemare@gmail.com>
> >> > > > > > >>>>> wrote:
> >> > > > > > >>>>>
> >> > > > > > >>>>>> Updated the PR to take into account John's feedback.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> I did some preliminary testing for the performance of
> the
> >> > > > > > prefixScan.
> >> > > > > > >> I
> >> > > > > > >>>>>> have attached the file, but I will also include the
> text
> >> in
> >> > > the
> >> > > > > body
> >> > > > > > >>>>> here
> >> > > > > > >>>>>> for archival purposes (I am not sure what happens to
> >> > attached
> >> > > > > > files).
> >> > > > > > >> I
> >> > > > > > >>>>>> also updated the PR and the KIP accordingly.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Summary: It scales exceptionally well for scanning
> large
> >> > > values
> >> > > > of
> >> > > > > > >>>>>> records. As Jan mentioned previously, the real issue
> >> would
> >> > be
> >> > > > more
> >> > > > > > >>>>> around
> >> > > > > > >>>>>> processing the resulting records after obtaining them.
> >> For
> >> > > > > instance,
> >> > > > > > >> it
> >> > > > > > >>>>>> takes approximately ~80-120 mS to flush the buffer and
> a
> >> > > further
> >> > > > > > >>>>> ~35-85mS
> >> > > > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of
> >> them.
> >> > > > > Iterating
> >> > > > > > >>>>>> through the records just to generate a simple count
> >> takes ~
> >> > 40
> >> > > > > times
> >> > > > > > >>>>> longer
> >> > > > > > >>>>>> than the flush + scan combined.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> ============================================================================================
> >> > > > > > >>>>>> Setup:
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> ============================================================================================
> >> > > > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
> >> > > (Xmx512m,
> >> > > > > > >> Xms512m)
> >> > > > > > >>>>>> CPU: i7 2.2 Ghz.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Note: I am using a slightly-modified,
> directly-accessible
> >> > > Kafka
> >> > > > > > >> Streams
> >> > > > > > >>>>>> RocksDB
> >> > > > > > >>>>>> implementation (RocksDB.java, basically just avoiding
> the
> >> > > > > > >>>>>> ProcessorContext).
> >> > > > > > >>>>>> There are no modifications to the default RocksDB
> values
> >> > > > provided
> >> > > > > in
> >> > > > > > >> the
> >> > > > > > >>>>>> 2.1/trunk release.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> keysize = 128 bytes
> >> > > > > > >>>>>> valsize = 512 bytes
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Step 1:
> >> > > > > > >>>>>> Write X positive matching events: (key = prefix +
> >> > left-padded
> >> > > > > > >>>>>> auto-incrementing integer)
> >> > > > > > >>>>>> Step 2:
> >> > > > > > >>>>>> Write 10X negative matching events (key = left-padded
> >> > > > > > >> auto-incrementing
> >> > > > > > >>>>>> integer)
> >> > > > > > >>>>>> Step 3:
> >> > > > > > >>>>>> Perform flush
> >> > > > > > >>>>>> Step 4:
> >> > > > > > >>>>>> Perform prefixScan
> >> > > > > > >>>>>> Step 5:
> >> > > > > > >>>>>> Iterate through return Iterator and validate the count
> of
> >> > > > expected
> >> > > > > > >>>>> events.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> ============================================================================================
> >> > > > > > >>>>>> Results:
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> ============================================================================================
> >> > > > > > >>>>>> X = 1k (11k events total)
> >> > > > > > >>>>>> Flush Time = 39 mS
> >> > > > > > >>>>>> Scan Time = 7 mS
> >> > > > > > >>>>>> 6.9 MB disk
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> --------------------------------------------------------------------------------------------
> >> > > > > > >>>>>> X = 10k (110k events total)
> >> > > > > > >>>>>> Flush Time = 45 mS
> >> > > > > > >>>>>> Scan Time = 8 mS
> >> > > > > > >>>>>> 127 MB
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> --------------------------------------------------------------------------------------------
> >> > > > > > >>>>>> X = 100k (1.1M events total)
> >> > > > > > >>>>>> Test1:
> >> > > > > > >>>>>> Flush Time = 60 mS
> >> > > > > > >>>>>> Scan Time = 12 mS
> >> > > > > > >>>>>> 678 MB
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Test2:
> >> > > > > > >>>>>> Flush Time = 45 mS
> >> > > > > > >>>>>> Scan Time = 7 mS
> >> > > > > > >>>>>> 576 MB
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> --------------------------------------------------------------------------------------------
> >> > > > > > >>>>>> X = 1MB (11M events total)
> >> > > > > > >>>>>> Test1:
> >> > > > > > >>>>>> Flush Time = 52 mS
> >> > > > > > >>>>>> Scan Time = 19 mS
> >> > > > > > >>>>>> 7.2 GB
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Test2:
> >> > > > > > >>>>>> Flush Time = 84 mS
> >> > > > > > >>>>>> Scan Time = 34 mS
> >> > > > > > >>>>>> 9.1 GB
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> --------------------------------------------------------------------------------------------
> >> > > > > > >>>>>> X = 2.5M (27.5M events total)
> >> > > > > > >>>>>> Test1:
> >> > > > > > >>>>>> Flush Time = 82 mS
> >> > > > > > >>>>>> Scan Time = 63 mS
> >> > > > > > >>>>>> 17GB - 276 sst files
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Test2:
> >> > > > > > >>>>>> Flush Time = 116 mS
> >> > > > > > >>>>>> Scan Time = 35 mS
> >> > > > > > >>>>>> 23GB - 361 sst files
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Test3:
> >> > > > > > >>>>>> Flush Time = 103 mS
> >> > > > > > >>>>>> Scan Time = 82 mS
> >> > > > > > >>>>>> 19 GB - 300 sst files
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> --------------------------------------------------------------------------------------------
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M
> >> events. I
> >> > > > tried
> >> > > > > > to
> >> > > > > > >> go
> >> > > > > > >>>>>> to X = 10M (110M events) but RocksDB was going into the
> >> > 100GB+
> >> > > > > range
> >> > > > > > >>>>> and my
> >> > > > > > >>>>>> laptop ran out of disk. More extensive testing could be
> >> done
> >> > > > but I
> >> > > > > > >>>>> suspect
> >> > > > > > >>>>>> that it would be in line with what we're seeing in the
> >> > results
> >> > > > > > above.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> At this point in time, I think the only major
> discussion
> >> > point
> >> > > > is
> >> > > > > > >> really
> >> > > > > > >>>>>> around what Jan and I have disagreed on: repartitioning
> >> > back +
> >> > > > > > >> resolving
> >> > > > > > >>>>>> potential out of order issues or leaving that up to the
> >> > client
> >> > > > to
> >> > > > > > >>>>> handle.
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Thanks folks,
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> Adam
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> >> > > > > > Jan.Filipiak@trivago.com
> >> > > > > > >>>
> >> > > > > > >>>>>> wrote:
> >> > > > > > >>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> >> > > > > > >>>>>>>> Hi all,
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Sorry that this discussion petered out... I think the
> >> 2.1
> >> > > > > release
> >> > > > > > >>>>>>> caused an
> >> > > > > > >>>>>>>> extended distraction that pushed it off everyone's
> >> radar
> >> > > > (which
> >> > > > > > was
> >> > > > > > >>>>>>>> precisely Adam's concern). Personally, I've also had
> >> some
> >> > > > extend
> >> > > > > > >>>>>>>> distractions of my own that kept (and continue to
> >> keep) me
> >> > > > > > >>>>> preoccupied.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> However, calling for a vote did wake me up, so I
> guess
> >> Jan
> >> > > was
> >> > > > > on
> >> > > > > > >> the
> >> > > > > > >>>>>>> right
> >> > > > > > >>>>>>>> track!
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> I've gone back and reviewed the whole KIP document
> and
> >> the
> >> > > > prior
> >> > > > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> API Thoughts:
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 1. If I read the KIP right, you are proposing a
> >> > many-to-one
> >> > > > > join.
> >> > > > > > >>>>> Could
> >> > > > > > >>>>>>> we
> >> > > > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
> >> flip
> >> > > the
> >> > > > > > design
> >> > > > > > >>>>>>> around
> >> > > > > > >>>>>>>> and make it a oneToManyJoin?
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the
> join
> >> > > type,
> >> > > > > and
> >> > > > > > it
> >> > > > > > >>>>>>> seems
> >> > > > > > >>>>>>>> like it might trick some people into using it for a
> >> > > one-to-one
> >> > > > > > join.
> >> > > > > > >>>>>>> This
> >> > > > > > >>>>>>>> would work, of course, but it would be super
> >> inefficient
> >> > > > > compared
> >> > > > > > to
> >> > > > > > >>>>> a
> >> > > > > > >>>>>>>> simple rekey-and-join.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 2. I might have missed it, but I don't think it's
> >> > specified
> >> > > > > > whether
> >> > > > > > >>>>>>> it's an
> >> > > > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer
> >> join, as
> >> > > > > > >>>>> (neglecting
> >> > > > > > >>>>>>> IQ),
> >> > > > > > >>>>>>>> the rest can be achieved by filtering or by handling
> >> it in
> >> > > the
> >> > > > > > >>>>>>> ValueJoiner.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look
> quite
> >> > > right.
> >> > > > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
> >> > > paradigms
> >> > > > in
> >> > > > > > >>>>> play in
> >> > > > > > >>>>>>>> the Streams API, so it's confusing, but instead of
> >> three
> >> > > > > > Serialized
> >> > > > > > >>>>>>> args, I
> >> > > > > > >>>>>>>> think it would be better to have one that allows
> >> > > (optionally)
> >> > > > > > >> setting
> >> > > > > > >>>>>>> the 4
> >> > > > > > >>>>>>>> incoming serdes. The result serde is defined by the
> >> > > > > Materialized.
> >> > > > > > >> The
> >> > > > > > >>>>>>>> incoming serdes can be optional because they might
> >> already
> >> > > be
> >> > > > > > >>>>> available
> >> > > > > > >>>>>>> on
> >> > > > > > >>>>>>>> the source KTables, or the default serdes from the
> >> config
> >> > > > might
> >> > > > > be
> >> > > > > > >>>>>>>> applicable.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other
> joins
> >> > > don't
> >> > > > > > allow
> >> > > > > > >>>>>>> setting
> >> > > > > > >>>>>>>> one, and it seems like it might actually be harmful,
> >> since
> >> > > the
> >> > > > > > rekey
> >> > > > > > >>>>>>>> operation needs to produce results that are
> >> co-partitioned
> >> > > > with
> >> > > > > > the
> >> > > > > > >>>>>>> "other"
> >> > > > > > >>>>>>>> KTable.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I
> >> didn't
> >> > > > > actually
> >> > > > > > >>>>>>> follow
> >> > > > > > >>>>>>>> what Matthias meant about namespacing requiring
> >> > > > "deserializing"
> >> > > > > > the
> >> > > > > > >>>>>>> record
> >> > > > > > >>>>>>>> header. The headers are already Strings, so I don't
> >> think
> >> > > that
> >> > > > > > >>>>>>>> deserialization is required. If we applied the
> >> namespace
> >> > at
> >> > > > > source
> >> > > > > > >>>>> nodes
> >> > > > > > >>>>>>>> and stripped it at sink nodes, this would be
> >> practically
> >> > no
> >> > > > > > >> overhead.
> >> > > > > > >>>>>>> The
> >> > > > > > >>>>>>>> advantage of the namespace idea is that no public API
> >> > change
> >> > > > wrt
> >> > > > > > >>>>> headers
> >> > > > > > >>>>>>>> needs to happen, and no restrictions need to be
> placed
> >> on
> >> > > > users'
> >> > > > > > >>>>>>> headers.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> (Although I'm wondering if we can get away without
> the
> >> > > header
> >> > > > at
> >> > > > > > >>>>> all...
> >> > > > > > >>>>>>>> stay tuned)
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM
> >> table
> >> > > > > growing
> >> > > > > > >>>>>>> without
> >> > > > > > >>>>>>>> bound. As I read it, the HWM table is effectively
> >> > > implementing
> >> > > > > OCC
> >> > > > > > >> to
> >> > > > > > >>>>>>>> resolve the problem you noted with disordering when
> the
> >> > > rekey
> >> > > > is
> >> > > > > > >>>>>>>> reversed... particularly notable when the FK changes.
> >> As
> >> > > such,
> >> > > > > it
> >> > > > > > >>>>> only
> >> > > > > > >>>>>>>> needs to track the most recent "version" (the offset
> in
> >> > the
> >> > > > > source
> >> > > > > > >>>>>>>> partition) of each key. Therefore, it should have the
> >> same
> >> > > > > number
> >> > > > > > of
> >> > > > > > >>>>>>> keys
> >> > > > > > >>>>>>>> as the source table at all times.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> I see that you are aware of KIP-258, which I think
> >> might
> >> > be
> >> > > > > > relevant
> >> > > > > > >>>>> in
> >> > > > > > >>>>>>> a
> >> > > > > > >>>>>>>> couple of ways. One: it's just about storing the
> >> timestamp
> >> > > in
> >> > > > > the
> >> > > > > > >>>>> state
> >> > > > > > >>>>>>>> store, but the ultimate idea is to effectively use
> the
> >> > > > timestamp
> >> > > > > > as
> >> > > > > > >>>>> an
> >> > > > > > >>>>>>> OCC
> >> > > > > > >>>>>>>> "version" to drop disordered updates. You wouldn't
> >> want to
> >> > > use
> >> > > > > the
> >> > > > > > >>>>>>>> timestamp for this operation, but if you were to use
> a
> >> > > similar
> >> > > > > > >>>>>>> mechanism to
> >> > > > > > >>>>>>>> store the source offset in the store alongside the
> >> > re-keyed
> >> > > > > > values,
> >> > > > > > >>>>> then
> >> > > > > > >>>>>>>> you could avoid a separate table.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 6. You and Jan have been thinking about this for a
> long
> >> > > time,
> >> > > > so
> >> > > > > > >> I've
> >> > > > > > >>>>>>>> probably missed something here, but I'm wondering if
> we
> >> > can
> >> > > > > avoid
> >> > > > > > >> the
> >> > > > > > >>>>>>> HWM
> >> > > > > > >>>>>>>> tracking at all and resolve out-of-order during a
> final
> >> > join
> >> > > > > > >>>>> instead...
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Let's say we're joining a left table (Integer K:
> Letter
> >> > FK,
> >> > > > > (other
> >> > > > > > >>>>>>> data))
> >> > > > > > >>>>>>>> to a right table (Letter K: (some data)).
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Left table:
> >> > > > > > >>>>>>>> 1: (A, xyz)
> >> > > > > > >>>>>>>> 2: (B, asd)
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Right table:
> >> > > > > > >>>>>>>> A: EntityA
> >> > > > > > >>>>>>>> B: EntityB
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> We could do a rekey as you proposed with a combined
> >> key,
> >> > but
> >> > > > not
> >> > > > > > >>>>>>>> propagating the value at all..
> >> > > > > > >>>>>>>> Rekey table:
> >> > > > > > >>>>>>>> A-1: (dummy value)
> >> > > > > > >>>>>>>> B-2: (dummy value)
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Which we then join with the right table to produce:
> >> > > > > > >>>>>>>> A-1: EntityA
> >> > > > > > >>>>>>>> B-2: EntityB
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Which gets rekeyed back:
> >> > > > > > >>>>>>>> 1: A, EntityA
> >> > > > > > >>>>>>>> 2: B, EntityB
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> And finally we do the actual join:
> >> > > > > > >>>>>>>> Result table:
> >> > > > > > >>>>>>>> 1: ((A, xyz), EntityA)
> >> > > > > > >>>>>>>> 2: ((B, asd), EntityB)
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> The thing is that in that last join, we have the
> >> > opportunity
> >> > > > to
> >> > > > > > >>>>> compare
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>> current FK in the left table with the incoming PK of
> >> the
> >> > > right
> >> > > > > > >>>>> table. If
> >> > > > > > >>>>>>>> they don't match, we just drop the event, since it
> >> must be
> >> > > > > > outdated.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz)
> >> gets
> >> > > > > updated
> >> > > > > > >> to
> >> > > > > > >>>>>>> (1:
> >> > > > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about
> whether
> >> the
> >> > > > final
> >> > > > > > >>>>> state
> >> > > > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the
> >> > algorithm
> >> > > > > above,
> >> > > > > > >>>>> you
> >> > > > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
> >> (B,
> >> > > xyz),
> >> > > > > (B,
> >> > > > > > >>>>>>>> EntityB)). It seems like this does give you enough
> >> > > information
> >> > > > > to
> >> > > > > > >>>>> make
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>> right choice, regardless of disordering.
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> Will check Adams patch, but this should work. As
> >> mentioned
> >> > > > often
> >> > > > > I
> >> > > > > > am
> >> > > > > > >>>>>>> not convinced on partitioning back for the user
> >> > > automatically.
> >> > > > I
> >> > > > > > >> think
> >> > > > > > >>>>>>> this is the real performance eater ;)
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
> >> > > > performance
> >> > > > > of
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>> range scans when records change in the right table.
> >> You've
> >> > > > said
> >> > > > > > that
> >> > > > > > >>>>>>> you've
> >> > > > > > >>>>>>>> been using the algorithm you presented in production
> >> for a
> >> > > > > while.
> >> > > > > > >> Can
> >> > > > > > >>>>>>> you
> >> > > > > > >>>>>>>> give us a sense of the performance characteristics
> >> you've
> >> > > > > > observed?
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> Make it work, make it fast, make it beautiful. The
> >> topmost
> >> > > > thing
> >> > > > > > here
> >> > > > > > >>>>> is
> >> > > > > > >>>>>>> / was correctness. In practice I do not measure the
> >> > > performance
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >>>>>>> range scan. Usual cases I run this with is emitting
> >> 500k -
> >> > > 1kk
> >> > > > > rows
> >> > > > > > >>>>>>> on a left hand side change. The range scan is just the
> >> work
> >> > > you
> >> > > > > > gotta
> >> > > > > > >>>>>>> do, also when you pack your data into different
> formats,
> >> > > > usually
> >> > > > > > the
> >> > > > > > >>>>>>> rocks performance is very tight to the size of the
> data
> >> and
> >> > > we
> >> > > > > > can't
> >> > > > > > >>>>>>> really change that. It is more important for users to
> >> > prevent
> >> > > > > > useless
> >> > > > > > >>>>>>> updates to begin with. My left hand side is guarded to
> >> drop
> >> > > > > changes
> >> > > > > > >>>>> that
> >> > > > > > >>>>>>> are not going to change my join output.
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> usually it's:
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> drop unused fields and then don't forward if
> >> > old.equals(new)
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>> regarding to the performance of creating an iterator
> for
> >> > > > smaller
> >> > > > > > >>>>>>> fanouts, users can still just do a group by first then
> >> > > anyways.
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>>> I could only think of one alternative, but I'm not
> >> sure if
> >> > > > it's
> >> > > > > > >>>>> better
> >> > > > > > >>>>>>> or
> >> > > > > > >>>>>>>> worse... If the first re-key only needs to preserve
> the
> >> > > > original
> >> > > > > > >> key,
> >> > > > > > >>>>>>> as I
> >> > > > > > >>>>>>>> proposed in #6, then we could store a vector of keys
> in
> >> > the
> >> > > > > value:
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Left table:
> >> > > > > > >>>>>>>> 1: A,...
> >> > > > > > >>>>>>>> 2: B,...
> >> > > > > > >>>>>>>> 3: A,...
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Gets re-keyed:
> >> > > > > > >>>>>>>> A: [1, 3]
> >> > > > > > >>>>>>>> B: [2]
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Then, the rhs part of the join would only need a
> >> regular
> >> > > > > > single-key
> >> > > > > > >>>>>>> lookup.
> >> > > > > > >>>>>>>> Of course we have to deal with the problem of large
> >> > values,
> >> > > as
> >> > > > > > >>>>> there's
> >> > > > > > >>>>>>> no
> >> > > > > > >>>>>>>> bound on the number of lhs records that can reference
> >> rhs
> >> > > > > records.
> >> > > > > > >>>>>>> Offhand,
> >> > > > > > >>>>>>>> I'd say we could page the values, so when one row is
> >> past
> >> > > the
> >> > > > > > >>>>>>> threshold, we
> >> > > > > > >>>>>>>> append the key for the next page. Then in most cases,
> >> it
> >> > > would
> >> > > > > be
> >> > > > > > a
> >> > > > > > >>>>>>> single
> >> > > > > > >>>>>>>> key lookup, but for large fan-out updates, it would
> be
> >> one
> >> > > per
> >> > > > > > (max
> >> > > > > > >>>>>>> value
> >> > > > > > >>>>>>>> size)/(avg lhs key size).
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> This seems more complex, though... Plus, I think
> >> there's
> >> > > some
> >> > > > > > extra
> >> > > > > > >>>>>>>> tracking we'd need to do to know when to emit a
> >> > retraction.
> >> > > > For
> >> > > > > > >>>>> example,
> >> > > > > > >>>>>>>> when record 1 is deleted, the re-key table would just
> >> have
> >> > > (A:
> >> > > > > > [3]).
> >> > > > > > >>>>>>> Some
> >> > > > > > >>>>>>>> kind of tombstone is needed so that the join result
> >> for 1
> >> > > can
> >> > > > > also
> >> > > > > > >> be
> >> > > > > > >>>>>>>> retracted.
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> That's all!
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> Thanks so much to both Adam and Jan for the
> thoughtful
> >> > KIP.
> >> > > > > Sorry
> >> > > > > > >> the
> >> > > > > > >>>>>>>> discussion has been slow.
> >> > > > > > >>>>>>>> -John
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> >> > > > > > >>>>> Jan.Filipiak@trivago.com>
> >> > > > > > >>>>>>>> wrote:
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>>> Id say you can just call the vote.
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>>> that happens all the time, and if something comes
> up,
> >> it
> >> > > just
> >> > > > > > goes
> >> > > > > > >>>>> back
> >> > > > > > >>>>>>>>> to discuss.
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>>> would not expect to much attention with another
> >> another
> >> > > email
> >> > > > > in
> >> > > > > > >>>>> this
> >> > > > > > >>>>>>>>> thread.
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>>> best Jan
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> >> > > > > > >>>>>>>>>> Hello Contributors
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do
> >> need
> >> > to
> >> > > > bump
> >> > > > > > >>>>> this to
> >> > > > > > >>>>>>>>> keep
> >> > > > > > >>>>>>>>>> visibility up. I am still intending to push this
> >> through
> >> > > > once
> >> > > > > > >>>>>>> contributor
> >> > > > > > >>>>>>>>>> feedback is given.
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> Main points that need addressing:
> >> > > > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
> >> > > singular
> >> > > > > > graph
> >> > > > > > >>>>> node
> >> > > > > > >>>>>>>>> into
> >> > > > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters
> right
> >> > > now. I
> >> > > > > am
> >> > > > > > a
> >> > > > > > >>>>> bit
> >> > > > > > >>>>>>>>> fuzzy
> >> > > > > > >>>>>>>>>> on how the optimizations are supposed to work, so I
> >> > would
> >> > > > > > >>>>> appreciate
> >> > > > > > >>>>>>> any
> >> > > > > > >>>>>>>>>> help on this aspect.
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This
> >> thread
> >> > > has
> >> > > > > > much
> >> > > > > > >>>>>>>>> discourse
> >> > > > > > >>>>>>>>>> between Jan and I between the current highwater
> mark
> >> > > > proposal
> >> > > > > > and
> >> > > > > > >> a
> >> > > > > > >>>>>>>>> groupBy
> >> > > > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need
> >> to
> >> > > > > strictly
> >> > > > > > >>>>> handle
> >> > > > > > >>>>>>>>> any
> >> > > > > > >>>>>>>>>> chance of out-of-order data and leave none of it up
> >> to
> >> > the
> >> > > > > > >>>>> consumer.
> >> > > > > > >>>>>>> Any
> >> > > > > > >>>>>>>>>> comments or suggestions here would also help.
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> 3) Anything else that you see that would prevent
> this
> >> > from
> >> > > > > > moving
> >> > > > > > >>>>> to a
> >> > > > > > >>>>>>>>> vote?
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> Thanks
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> Adam
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >> > > > > > >>>>>>>>> adam.bellemare@gmail.com>
> >> > > > > > >>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>> Hi Jan
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> >> > > > > > >>>>> Stores.persistentWindowStore,
> >> > > > > > >>>>>>> you
> >> > > > > > >>>>>>>>>>> actually only need to specify the amount of
> segments
> >> > you
> >> > > > want
> >> > > > > > and
> >> > > > > > >>>>> how
> >> > > > > > >>>>>>>>> large
> >> > > > > > >>>>>>>>>>> they are. To the best of my understanding, what
> >> happens
> >> > > is
> >> > > > > that
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>>>> segments are automatically rolled over as new data
> >> with
> >> > > new
> >> > > > > > >>>>>>> timestamps
> >> > > > > > >>>>>>>>> are
> >> > > > > > >>>>>>>>>>> created. We use this exact functionality in some
> of
> >> the
> >> > > > work
> >> > > > > > done
> >> > > > > > >>>>>>>>>>> internally at my company. For reference, this is
> the
> >> > > > hopping
> >> > > > > > >>>>> windowed
> >> > > > > > >>>>>>>>> store.
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>> In the code that I have provided, there are going
> >> to be
> >> > > two
> >> > > > > 24h
> >> > > > > > >>>>>>>>> segments.
> >> > > > > > >>>>>>>>>>> When a record is put into the windowStore, it will
> >> be
> >> > > > > inserted
> >> > > > > > at
> >> > > > > > >>>>>>> time
> >> > > > > > >>>>>>>>> T in
> >> > > > > > >>>>>>>>>>> both segments. The two segments will always
> overlap
> >> by
> >> > > 12h.
> >> > > > > As
> >> > > > > > >>>>> time
> >> > > > > > >>>>>>>>> goes on
> >> > > > > > >>>>>>>>>>> and new records are added (say at time T+12h+),
> the
> >> > > oldest
> >> > > > > > >> segment
> >> > > > > > >>>>>>> will
> >> > > > > > >>>>>>>>> be
> >> > > > > > >>>>>>>>>>> automatically deleted and a new segment created.
> The
> >> > > > records
> >> > > > > > are
> >> > > > > > >>>>> by
> >> > > > > > >>>>>>>>> default
> >> > > > > > >>>>>>>>>>> inserted with the context.timestamp(), such that
> it
> >> is
> >> > > the
> >> > > > > > record
> >> > > > > > >>>>>>> time,
> >> > > > > > >>>>>>>>> not
> >> > > > > > >>>>>>>>>>> the clock time, which is used.
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>> To the best of my understanding, the timestamps
> are
> >> > > > retained
> >> > > > > > when
> >> > > > > > >>>>>>>>>>> restoring from the changelog.
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with
> TTL
> >> > at a
> >> > > > > > >>>>>>> segment-level,
> >> > > > > > >>>>>>>>>>> instead of at an individual record level.
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >> > > > > > >>>>>>> Jan.Filipiak@trivago.com>
> >> > > > > > >>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
> >> > > > > > ClassCastException
> >> > > > > > >>>>> or
> >> > > > > > >>>>>>>>>>>> similar.
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> You either would have to specify the window you
> >> > > fetch/put
> >> > > > or
> >> > > > > > >>>>> iterate
> >> > > > > > >>>>>>>>>>>> across all windows the key was found in right?
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> I just hope the window-store doesn't check
> >> stream-time
> >> > > > under
> >> > > > > > the
> >> > > > > > >>>>>>> hoods
> >> > > > > > >>>>>>>>>>>> that would be a questionable interface.
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> If it does: did you see my comment on checking
> all
> >> the
> >> > > > > windows
> >> > > > > > >>>>>>> earlier?
> >> > > > > > >>>>>>>>>>>> that would be needed to actually give reasonable
> >> time
> >> > > > > > gurantees.
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> Best
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >> > > > > > >>>>>>>>>>>>> Hi Jan
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
> >> changed
> >> > > the
> >> > > > > > state
> >> > > > > > >>>>>>> store,
> >> > > > > > >>>>>>>>>>>> not
> >> > > > > > >>>>>>>>>>>>> the ProcessorSupplier.
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>> Thanks,
> >> > > > > > >>>>>>>>>>>>> Adam
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> @Guozhang
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed
> >> > something
> >> > > > that
> >> > > > > > >>>>> will be
> >> > > > > > >>>>>>>>>>>>>>> extremely
> >> > > > > > >>>>>>>>>>>>>>> useful for this KIP.
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> @Jan
> >> > > > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being
> said, I
> >> > will
> >> > > > not
> >> > > > > > be
> >> > > > > > >>>>>>> moving
> >> > > > > > >>>>>>>>>>>> ahead
> >> > > > > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy
> >> > > solution
> >> > > > > as
> >> > > > > > >> you
> >> > > > > > >>>>>>>>>>>> propose.
> >> > > > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it
> >> > yourself
> >> > > > off
> >> > > > > > of
> >> > > > > > >>>>> my
> >> > > > > > >>>>>>>>>>>> current PR
> >> > > > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I
> >> would
> >> > > be
> >> > > > > more
> >> > > > > > >>>>> than
> >> > > > > > >>>>>>>>>>>> happy to
> >> > > > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> >> > stands
> >> > > > > right
> >> > > > > > >>>>> now,
> >> > > > > > >>>>>>> I do
> >> > > > > > >>>>>>>>>>>> not
> >> > > > > > >>>>>>>>>>>>>>> really have more time to invest into
> >> alternatives
> >> > > > without
> >> > > > > > >>>>> there
> >> > > > > > >>>>>>>>> being
> >> > > > > > >>>>>>>>>>>> a
> >> > > > > > >>>>>>>>>>>>>>> strong indication from the binding voters
> which
> >> > they
> >> > > > > would
> >> > > > > > >>>>>>> prefer.
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally
> gave
> >> up
> >> > on
> >> > > > the
> >> > > > > > >>>>> streams
> >> > > > > > >>>>>>>>> DSL
> >> > > > > > >>>>>>>>>>>> for
> >> > > > > > >>>>>>>>>>>>>> some time already, otherwise I would have
> pulled
> >> > this
> >> > > > KIP
> >> > > > > > >>>>> through
> >> > > > > > >>>>>>>>>>>> already.
> >> > > > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based
> on
> >> > > PAPI.
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the
> >> windowed
> >> > > > state
> >> > > > > > >>>>> store
> >> > > > > > >>>>>>> in
> >> > > > > > >>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>> next
> >> > > > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then
> I
> >> > will
> >> > > > come
> >> > > > > > >> back
> >> > > > > > >>>>>>> for
> >> > > > > > >>>>>>>>>>>> final
> >> > > > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any
> of
> >> > the
> >> > > > > > binding
> >> > > > > > >>>>>>> voters
> >> > > > > > >>>>>>>>>>>> could
> >> > > > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
> >> updated
> >> > it
> >> > > > > > >>>>> according
> >> > > > > > >>>>>>> to
> >> > > > > > >>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>> latest plan:
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >> > > > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a
> windowed
> >> > > store.
> >> > > > > > This
> >> > > > > > >>>>>>> could
> >> > > > > > >>>>>>>>> be
> >> > > > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever
> they
> >> > are
> >> > > > > > >>>>> completed.
> >> > > > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> Thanks,
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> Adam
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> >> > already
> >> > > > > > updated
> >> > > > > > >>>>> in
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> PR?
> >> > > > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long
> Missing
> >> > > > > something?
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang
> Wang <
> >> > > > > > >>>>>>> wangguoz@gmail.com>
> >> > > > > > >>>>>>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is
> >> the
> >> > > > wrong
> >> > > > > > >> link,
> >> > > > > > >>>>>>> as it
> >> > > > > > >>>>>>>>>>>> is
> >> > > > > > >>>>>>>>>>>>>>>> for
> >> > > > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
> >> part of
> >> > > > > KIP-258
> >> > > > > > >>>>> we do
> >> > > > > > >>>>>>>>>>>> want to
> >> > > > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source
> >> > KTable"
> >> > > > such
> >> > > > > > >> that
> >> > > > > > >>>>>>>>>>>> instead of
> >> > > > > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized
> >> > store,
> >> > > > > i.e.
> >> > > > > > >>>>>>> following
> >> > > > > > >>>>>>>>>>>>>>>> offset
> >> > > > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are
> older
> >> > than
> >> > > > the
> >> > > > > > >>>>> current
> >> > > > > > >>>>>>>>>>>> key's
> >> > > > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp
> ordering.
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>> Guozhang
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
> >> Wang <
> >> > > > > > >>>>>>>>> wangguoz@gmail.com>
> >> > > > > > >>>>>>>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>> Hello Adam,
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
> >> final
> >> > > step
> >> > > > > > (i.e.
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>> high
> >> > > > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
> >> with
> >> > a
> >> > > > > window
> >> > > > > > >>>>>>> store),
> >> > > > > > >>>>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>> think
> >> > > > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually
> >> help:
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a
> >> key-value
> >> > > > store
> >> > > > > > >>>>> (i.e.
> >> > > > > > >>>>>>> only
> >> > > > > > >>>>>>>>>>>> for
> >> > > > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
> >> usage,
> >> > as
> >> > > > > > >>>>> described
> >> > > > > > >>>>>>> in
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> https://issues.apache.org/jira/browse/KAFKA-5533
> >> > ,
> >> > > is
> >> > > > > > that
> >> > > > > > >>>>> we
> >> > > > > > >>>>>>> can
> >> > > > > > >>>>>>>>>>>> then
> >> > > > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if
> its
> >> > > > > timestamp
> >> > > > > > is
> >> > > > > > >>>>>>>>> smaller
> >> > > > > > >>>>>>>>>>>> than
> >> > > > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I
> >> > think
> >> > > it
> >> > > > > is
> >> > > > > > >>>>> very
> >> > > > > > >>>>>>>>>>>> similar to
> >> > > > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark
> based
> >> > > > > filtering,
> >> > > > > > >>>>> while
> >> > > > > > >>>>>>>>> you
> >> > > > > > >>>>>>>>>>>> only
> >> > > > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the
> >> > > joining
> >> > > > > > >> records
> >> > > > > > >>>>>>> are
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>> correctly
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the
> >> final
> >> > > > stage.
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store
> and
> >> > hence
> >> > > > > > >>>>>>> non-windowed
> >> > > > > > >>>>>>>>>>>> KTables
> >> > > > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
> >> really
> >> > > have
> >> > > > a
> >> > > > > > good
> >> > > > > > >>>>>>>>> support
> >> > > > > > >>>>>>>>>>>> for
> >> > > > > > >>>>>>>>>>>>>>>>> their joins anyways (
> >> > > > > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107
> )
> >> > > > > > >>>>>>>>>>>>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
> >> > > KTable-KTable
> >> > > > > > >>>>> non-key
> >> > > > > > >>>>>>>>> joins
> >> > > > > > >>>>>>>>>>>> for
> >> > > > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> Guozhang
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan
> Filipiak
> >> <
> >> > > > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation
> would
> >> > grow
> >> > > > > > >> endlessly
> >> > > > > > >>>>>>> based
> >> > > > > > >>>>>>>>>>>> on
> >> > > > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a
> pair
> >> of
> >> > > > (<this
> >> > > > > > >>>>> table
> >> > > > > > >>>>>>>>>>>> primary
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> key>,
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is
> >> used
> >> > > to
> >> > > > > > >>>>>>> differentiate
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> between
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
> >> proposal
> >> > > > would
> >> > > > > > be
> >> > > > > > >>>>> to
> >> > > > > > >>>>>>>>>>>> replace
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> it
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
> >> This
> >> > > would
> >> > > > > > allow
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>> same
> >> > > > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time.
> >> This
> >> > > > > should
> >> > > > > > >>>>> allow
> >> > > > > > >>>>>>> for
> >> > > > > > >>>>>>>>>>>> all
> >> > > > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> >> > should
> >> > > be
> >> > > > > > >>>>>>> customizable
> >> > > > > > >>>>>>>>>>>> by
> >> > > > > > >>>>>>>>>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
> >> perhaps
> >> > > just
> >> > > > > 10
> >> > > > > > >>>>>>> minutes
> >> > > > > > >>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> window,
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do
> >> the
> >> > > > trick
> >> > > > > > >> here.
> >> > > > > > >>>>>>> Even
> >> > > > > > >>>>>>>>>>>> if I
> >> > > > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
> >> > > repartitioning
> >> > > > > > >>>>> optional
> >> > > > > > >>>>>>>>>>>> since I
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
> >> am a
> >> > > > little
> >> > > > > > bit
> >> > > > > > >>>>>>>>>>>> sceptical
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> about
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially
> one
> >> > > could
> >> > > > > run
> >> > > > > > >>>>> into
> >> > > > > > >>>>>>>>>>>> problems
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> when
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window
> >> border. I
> >> > > will
> >> > > > > > check
> >> > > > > > >>>>> you
> >> > > > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its
> >> problematic, we
> >> > > > could
> >> > > > > > >>>>> still
> >> > > > > > >>>>>>>>> check
> >> > > > > > >>>>>>>>>>>>>>>>>> _all_
> >> > > > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance
> >> > > impact I
> >> > > > > > >> guess.
> >> > > > > > >>>>>>> Will
> >> > > > > > >>>>>>>>>>>> let
> >> > > > > > >>>>>>>>>>>>>>>>>> you
> >> > > > > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct
> >> as
> >> > > is. I
> >> > > > > > >>>>> wouldn't
> >> > > > > > >>>>>>> not
> >> > > > > > >>>>>>>>>>>> like
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> >> > > timestamp(A)  <
> >> > > > > > >>>>>>>>> timestamp(B).
> >> > > > > > >>>>>>>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> think
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> we can't expect that.
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> @Jan
> >> > > > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now -
> >> > thanks
> >> > > > for
> >> > > > > > the
> >> > > > > > >>>>>>>>>>>> diagram, it
> >> > > > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do
> >> not
> >> > > have
> >> > > > > the
> >> > > > > > >>>>>>> original
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> primary
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
> >> > > available
> >> > > > > > then
> >> > > > > > >>>>> you
> >> > > > > > >>>>>>>>>>>> would be
> >> > > > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the
> Map.
> >> > That
> >> > > > > being
> >> > > > > > >>>>> said,
> >> > > > > > >>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> encourage
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just
> for
> >> > > clarity
> >> > > > > for
> >> > > > > > >>>>>>> everyone
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> else.
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really
> >> hard
> >> > > > work.
> >> > > > > > But
> >> > > > > > >>>>> I
> >> > > > > > >>>>>>>>>>>> understand
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> >> > > original
> >> > > > > > >> primary
> >> > > > > > >>>>>>> key,
> >> > > > > > >>>>>>>>> We
> >> > > > > > >>>>>>>>>>>>>>>>>> have
> >> > > > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in
> PAPI
> >> > and
> >> > > > > > >> basically
> >> > > > > > >>>>>>> not
> >> > > > > > >>>>>>>>>>>> using
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> any
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely
> missed
> >> > that
> >> > > in
> >> > > > > > >>>>> original
> >> > > > > > >>>>>>> DSL
> >> > > > > > >>>>>>>>>>>> its
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> not
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
> >> up on
> >> > > my
> >> > > > > end.
> >> > > > > > >>>>> Will
> >> > > > > > >>>>>>>>>>>> finish
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
> >> week.
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the
> >> Map
> >> > > stay
> >> > > > > > >> inside
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> State
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the
> changes
> >> > have
> >> > > > > > >>>>> propagated?
> >> > > > > > >>>>>>>>> Isn't
> >> > > > > > >>>>>>>>>>>>>>>>>>> this
> >> > > > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
> >> state
> >> > > > store?
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
> >> substractor
> >> > is
> >> > > > > gonna
> >> > > > > > >>>>>>> return
> >> > > > > > >>>>>>>>>>>> `null`
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
> >> there
> >> > is
> >> > > > > going
> >> > > > > > to
> >> > > > > > >>>>> be
> >> > > > > > >>>>>>> a
> >> > > > > > >>>>>>>>>>>> store
> >> > > > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this
> >> > store
> >> > > > > > directly
> >> > > > > > >>>>> for
> >> > > > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues()
> is a
> >> > > > regular
> >> > > > > > >>>>> store,
> >> > > > > > >>>>>>>>>>>> satisfying
> >> > > > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
> >> join.
> >> > > The
> >> > > > > > >>>>> Windowed
> >> > > > > > >>>>>>>>>>>> store is
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> not
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next
> statefull
> >> > > > operation
> >> > > > > > we
> >> > > > > > >>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
> >> have
> >> > the
> >> > > > > > window
> >> > > > > > >>>>>>> store
> >> > > > > > >>>>>>>>>>>> also
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> have
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> the values then.
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a
> custom
> >> > group
> >> > > > by
> >> > > > > > >>>>> before
> >> > > > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key
> i
> >> > think
> >> > > > it
> >> > > > > > >> would
> >> > > > > > >>>>>>> help
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> users
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given
> the
> >> > > > original
> >> > > > > > >>>>> primary
> >> > > > > > >>>>>>>>> key
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> issue I
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid
> >> foundation
> >> > > to
> >> > > > > > build
> >> > > > > > >>>>> on.
> >> > > > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the
> user.
> >> > very
> >> > > > > > >>>>>>> unfortunate. I
> >> > > > > > >>>>>>>>>>>> could
> >> > > > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I
> do
> >> not
> >> > > > think
> >> > > > > > its
> >> > > > > > >>>>> a
> >> > > > > > >>>>>>> good
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> decision.
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> Thanks
> >> > > > > > >>>>>>>>>>>>>>>>>>> Adam
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta
> >> > > Dumbre <
> >> > > > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >> > > > > > >>>>>>> dumbreprajakta311@gmail.com
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM
> >> Jan
> >> > > > > Filipiak
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
> >> <mailto:
> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make
> >> such a
> >> > > > > chart.
> >> > > > > > >> last
> >> > > > > > >>>>>>> time i
> >> > > > > > >>>>>>>>>>>> didn't
> >> > > > > > >>>>>>>>>>>>>>>>>>>           get along
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all
> >> your
> >> > > > > charts.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done
> >> today
> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam
> >> Bellemare
> >> > > > > wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of
> >> > what I
> >> > > > > > >> attempted
> >> > > > > > >>>>> on
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> KIP.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>
> >> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>
> >> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >> > > > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <
> >> > > > > > >>>>>>>>>>>>
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>
> >> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >> > > > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the
> >> > start
> >> > > of
> >> > > > > my
> >> > > > > > own
> >> > > > > > >>>>>>>>>>>> implementation
> >> > > > > > >>>>>>>>>>>>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           this
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could
> >> not
> >> > get
> >> > > > it
> >> > > > > to
> >> > > > > > >>>>> work I
> >> > > > > > >>>>>>> have
> >> > > > > > >>>>>>>>>>>> since
> >> > > > > > >>>>>>>>>>>>>>>>>>>           discarded the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time,
> >> if
> >> > you
> >> > > > > wish
> >> > > > > > to
> >> > > > > > >>>>>>> continue
> >> > > > > > >>>>>>>>>>>> pursuing
> >> > > > > > >>>>>>>>>>>>>>>>>>>           for your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that
> >> you
> >> > > > please
> >> > > > > > >>>>> create a
> >> > > > > > >>>>>>>>>>>> diagram on
> >> > > > > > >>>>>>>>>>>>>>>>>>>           the KIP
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your
> >> > solution.
> >> > > > > > Please
> >> > > > > > >>>>> feel
> >> > > > > > >>>>>>> free
> >> > > > > > >>>>>>>>> to
> >> > > > > > >>>>>>>>>>>> use
> >> > > > > > >>>>>>>>>>>>>>>>>>>           the image I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting
> >> point.
> >> > I
> >> > > am
> >> > > > > > having
> >> > > > > > >>>>>>> trouble
> >> > > > > > >>>>>>>>>>>>>>>>>>>           understanding your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think
> that
> >> a
> >> > > > > carefully
> >> > > > > > >>>>>>> constructed
> >> > > > > > >>>>>>>>>>>> diagram
> >> > > > > > >>>>>>>>>>>>>>>>>>>           will clear
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > up
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings.
> >> > Alternately,
> >> > > > > > please
> >> > > > > > >>>>> post a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only
> >> guess
> >> > at
> >> > > > what
> >> > > > > > you
> >> > > > > > >>>>>>> mean, and
> >> > > > > > >>>>>>>>>>>> since I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           value my
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > own
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value
> >> yours,
> >> > I
> >> > > > > > believe
> >> > > > > > >> it
> >> > > > > > >>>>> is
> >> > > > > > >>>>>>> your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           responsibility to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation
> >> instead
> >> > > of
> >> > > > me
> >> > > > > > >>>>> trying to
> >> > > > > > >>>>>>>>> guess.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > Adam
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00
> >> AM,
> >> > Jan
> >> > > > > > Filipiak
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
> >> <mailto:
> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing
> >> > interested.
> >> > > > > kafka
> >> > > > > > >>>>>>> streams at
> >> > > > > > >>>>>>>>>>>> this
> >> > > > > > >>>>>>>>>>>>>>>>>>>           point supports
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long
> as
> >> > both
> >> > > > > > streams
> >> > > > > > >>>>> have
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> same
> >> > > > > > >>>>>>>>>>>>>>>>>>> key.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently
> >> implementing a
> >> > > > join
> >> > > > > > >> where a
> >> > > > > > >>>>>>> KTable
> >> > > > > > >>>>>>>>>>>> and a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           KTable can
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship
> >> > (1:n).
> >> > > > We
> >> > > > > > >> exploit
> >> > > > > > >>>>>>> that
> >> > > > > > >>>>>>>>>>>> rocksdb
> >> > > > > > >>>>>>>>>>>>>>>>>>> is
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> a
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data
> >> sorted
> >> > (At
> >> > > > > least
> >> > > > > > >>>>>>> exposes an
> >> > > > > > >>>>>>>>>>>> API to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           access the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted
> >> > fashion).
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical
> caveats
> >> > are
> >> > > > well
> >> > > > > > >>>>>>> understood
> >> > > > > > >>>>>>>>> now
> >> > > > > > >>>>>>>>>>>> and we
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> are
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > basically
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API
> >> > Design
> >> > > (
> >> > > > > when
> >> > > > > > >> Adam
> >> > > > > > >>>>>>> sees
> >> > > > > > >>>>>>>>> my
> >> > > > > > >>>>>>>>>>>> newest
> >> > > > > > >>>>>>>>>>>>>>>>>>>           message).
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track
> >> record of
> >> > > > > loosing
> >> > > > > > >>>>> those
> >> > > > > > >>>>>>> kinda
> >> > > > > > >>>>>>>>>>>>>>>>>>>           arguments within
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I
> have
> >> no
> >> > > clue
> >> > > > > > why.
> >> > > > > > >> So
> >> > > > > > >>>>> I
> >> > > > > > >>>>>>>>>>>> literally
> >> > > > > > >>>>>>>>>>>>>>>>>>>           can't wait for
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > you
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this
> thread
> >> and
> >> > > > give
> >> > > > > > you
> >> > > > > > >>>>>>> opinion on
> >> > > > > > >>>>>>>>>>>> how we
> >> > > > > > >>>>>>>>>>>>>>>>>>>           should
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > design
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
> >> > > oneToManyJoin
> >> > > > > and
> >> > > > > > >> how
> >> > > > > > >>>>>>> many
> >> > > > > > >>>>>>>>>>>> power we
> >> > > > > > >>>>>>>>>>>>>>>>>>>           want to give
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity"
> >> (where
> >> > > > > > simplicity
> >> > > > > > >>>>> isn't
> >> > > > > > >>>>>>>>>>>> really that
> >> > > > > > >>>>>>>>>>>>>>>>>>>           as users
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > still
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I
> >> argue)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in
> on
> >> > the
> >> > > > > > >> discussion
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James
> >> Kwan
> >> > > > wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group
> and I
> >> > > found
> >> > > > > this
> >> > > > > > >>>>> subject
> >> > > > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > like
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to
> implement a
> >> > join
> >> > > > > > table of
> >> > > > > > >>>>> two
> >> > > > > > >>>>>>>>>>>> streams? Is
> >> > > > > > >>>>>>>>>>>>>>>>>>> there
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > somewhere
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
> >> > > requirement
> >> > > > or
> >> > > > > > >>>>> proposal?
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13
> AM,
> >> Jan
> >> > > > > > Filipiak
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
> >> <mailto:
> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
> >> > > > Bellemare
> >> > > > > > >> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing
> >> using a
> >> > > > > > Windowed
> >> > > > > > >>>>> Store
> >> > > > > > >>>>>>> to
> >> > > > > > >>>>>>>>>>>> store the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           highwater
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this
> >> > should
> >> > > > work
> >> > > > > > >> fine,
> >> > > > > > >>>>>>> with
> >> > > > > > >>>>>>>>> the
> >> > > > > > >>>>>>>>>>>> caveat
> >> > > > > > >>>>>>>>>>>>>>>>>>>           being that
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > it
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve
> out-of-order
> >> > > arrival
> >> > > > > > for up
> >> > > > > > >>>>> to
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> size of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           the window
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > (ie:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This
> would
> >> > > remove
> >> > > > > the
> >> > > > > > >>>>>>> possibility
> >> > > > > > >>>>>>>>>>>> of it
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> being
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > unbounded
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
> >> > > > suggestion, I
> >> > > > > > >>>>> believe
> >> > > > > > >>>>>>> this
> >> > > > > > >>>>>>>>> is
> >> > > > > > >>>>>>>>>>>> where
> >> > > > > > >>>>>>>>>>>>>>>>>>>           we will
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement.
> >> > While I
> >> > > > do
> >> > > > > > not
> >> > > > > > >>>>>>> disagree
> >> > > > > > >>>>>>>>>>>> with your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           statement
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be
> >> additional
> >> > > > joins
> >> > > > > > done
> >> > > > > > >>>>> in a
> >> > > > > > >>>>>>>>>>>> real-world
> >> > > > > > >>>>>>>>>>>>>>>>>>>           workflow, I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > do
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can
> >> conclusively
> >> > > deal
> >> > > > > > with
> >> > > > > > >>>>>>>>> out-of-order
> >> > > > > > >>>>>>>>>>>>>>>>>>> arrival
> >> > > > > > >>>>>>>>>>>>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent
> >> > joins. I
> >> > > > > have
> >> > > > > > >>>>>>> attempted
> >> > > > > > >>>>>>>>> what
> >> > > > > > >>>>>>>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           think you have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a
> >> > high-water,
> >> > > > > using
> >> > > > > > >>>>>>> groupBy and
> >> > > > > > >>>>>>>>>>>> reduce)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           and found
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes
> >> too
> >> > > > > quickly,
> >> > > > > > or
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>> load
> >> > > > > > >>>>>>>>> on
> >> > > > > > >>>>>>>>>>>> a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           stream thread
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > is
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined
> messages
> >> > will
> >> > > > > arrive
> >> > > > > > >>>>>>>>> out-of-order
> >> > > > > > >>>>>>>>>>>> and be
> >> > > > > > >>>>>>>>>>>>>>>>>>>           incorrectly
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> >> > > > > intermediate
> >> > > > > > >>>>> event
> >> > > > > > >>>>>>> is
> >> > > > > > >>>>>>>>>>>>>>>>>>> represented
> >> > > > > > >>>>>>>>>>>>>>>>>>>           as the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > final
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light
> on
> >> > your
> >> > > > > > groupBy
> >> > > > > > >>>>>>>>>>>> implementation.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           There must be
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where
> it
> >> > is,
> >> > > I
> >> > > > > > would
> >> > > > > > >>>>> just
> >> > > > > > >>>>>>> like
> >> > > > > > >>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it
> >> must be
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess
> up.
> >> I
> >> > > would
> >> > > > > > like
> >> > > > > > >> to
> >> > > > > > >>>>>>> clarify
> >> > > > > > >>>>>>>>>>>> before
> >> > > > > > >>>>>>>>>>>>>>>>>>>           we draw a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the
> >> > scattered
> >> > > > > events
> >> > > > > > >>>>> back to
> >> > > > > > >>>>>>>>> their
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> original
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only
> way I
> >> > know
> >> > > > how
> >> > > > > > to
> >> > > > > > >>>>>>>>> conclusively
> >> > > > > > >>>>>>>>>>>> deal
> >> > > > > > >>>>>>>>>>>>>>>>>>>           with
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a
> >> > given
> >> > > > time
> >> > > > > > >> frame,
> >> > > > > > >>>>>>> and to
> >> > > > > > >>>>>>>>>>>> ensure
> >> > > > > > >>>>>>>>>>>>>>>>>>>           that the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > data
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent
> with
> >> > the
> >> > > > > input
> >> > > > > > >>>>> events.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to
> >> > share
> >> > > > that
> >> > > > > > >>>>>>> illustrates
> >> > > > > > >>>>>>>>> your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           approach, I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > would
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it
> would
> >> > > remove
> >> > > > > any
> >> > > > > > >>>>>>>>>>>> misunderstandings
> >> > > > > > >>>>>>>>>>>>>>>>>>>           that I may
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking
> >> for
> >> > my
> >> > > > > code.
> >> > > > > > I
> >> > > > > > >>>>> don't
> >> > > > > > >>>>>>> have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           something easily
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its
> >> bloated
> >> > > with
> >> > > > > > >>>>> OO-patterns.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K
> >> aggKey,
> >> > V
> >> > > > > > value, T
> >> > > > > > >>>>>>>>> aggregate)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> >> > > > > currentStateAsMap =
> >> > > > > > >>>>>>>>>>>> asMap(aggregate);
> >> > > > > > >>>>>>>>>>>>>>>>>>> <<
> >> > > > > > >>>>>>>>>>>>>>>>>>>           imaginary
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> >> > > > > > >>>>> mapper.apply(value);
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is
> the
> >> > > place
> >> > > > > > where
> >> > > > > > >>>>> people
> >> > > > > > >>>>>>>>>>>> actually
> >> > > > > > >>>>>>>>>>>>>>>>>>>           gonna have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > issues
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably
> >> couldn't
> >> > do
> >> > > > it.
> >> > > > > > we
> >> > > > > > >>>>> would
> >> > > > > > >>>>>>> need
> >> > > > > > >>>>>>>>>>>> to find
> >> > > > > > >>>>>>>>>>>>>>>>>>>           a solution
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > here.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that
> yet.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we
> >> propagate
> >> > the
> >> > > > > > field in
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>>>>> joiner, so
> >> > > > > > >>>>>>>>>>>>>>>>>>>           that we can
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > pick
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate.
> >> > Probably
> >> > > > you
> >> > > > > > have
> >> > > > > > >>>>> not
> >> > > > > > >>>>>>>>> thought
> >> > > > > > >>>>>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           this in your
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very
> >> open
> >> > > to
> >> > > > > > find a
> >> > > > > > >>>>>>> generic
> >> > > > > > >>>>>>>>>>>> solution
> >> > > > > > >>>>>>>>>>>>>>>>>>>           here. In my
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is
> >> broken
> >> > in
> >> > > > > > >>>>>>>>> KTableImpl.GroupBy
> >> > > > > > >>>>>>>>>>>> that
> >> > > > > > >>>>>>>>>>>>>>>>>>> it
> >> > > > > > >>>>>>>>>>>>>>>>>>>           looses
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the keys
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the
> >> > aggregate
> >> > > > > key.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I
> >> abstracted
> >> > it
> >> > > > away
> >> > > > > > back
> >> > > > > > >>>>>>> then way
> >> > > > > > >>>>>>>>>>>> before
> >> > > > > > >>>>>>>>>>>>>>>>>>> i
> >> > > > > > >>>>>>>>>>>>>>>>>>> was
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > thinking
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That
> is
> >> > why I
> >> > > > > > didn't
> >> > > > > > >>>>>>> realize
> >> > > > > > >>>>>>>>> its
> >> > > > > > >>>>>>>>>>>>>>>>>>>           significance here.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m :
> >> current)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > currentStateAsMap.put(mapper.apply(m),
> >> > > > > > >> m);
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > currentStateAsMap.put(toModifyKey,
> >> > > > > > >> value);
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > currentStateAsMap.remove(toModifyKey);
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > if(currentStateAsMap.isEmpty()){
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return
> >> null;
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> >> > > > > > >>>>>>> asAggregateType(currentStateAsMap)
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at
> >> 3:35
> >> > PM,
> >> > > > Jan
> >> > > > > > >>>>> Filipiak
> >> > > > > > >>>>>>> <
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com
> >> <mailto:
> >> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing
> >> > > Matthias
> >> > > > > to
> >> > > > > > >>>>> speed!
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences.
> I
> >> > think
> >> > > > > > >> re-keying
> >> > > > > > >>>>>>> back
> >> > > > > > >>>>>>>>>>>> should be
> >> > > > > > >>>>>>>>>>>>>>>>>>>           optional at
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> >> > > > > > KScatteredTable
> >> > > > > > >>>>> with
> >> > > > > > >>>>>>>>>>>> reshuffle()
> >> > > > > > >>>>>>>>>>>>>>>>>>>           returning
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> KTable<originalKey,Joined>
> >> > to
> >> > > > make
> >> > > > > > the
> >> > > > > > >>>>>>> backwards
> >> > > > > > >>>>>>>>>>>>>>>>>>>           repartitioning
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big
> >> favour of
> >> > > > doing
> >> > > > > > the
> >> > > > > > >>>>> out
> >> > > > > > >>>>>>> of
> >> > > > > > >>>>>>>>> order
> >> > > > > > >>>>>>>>>>>>>>>>>>>           processing using
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water
> >> mark
> >> > > > > tracking.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded
> >> > growth
> >> > > is
> >> > > > > > just
> >> > > > > > >>>>> scary
> >> > > > > > >>>>>>> + It
> >> > > > > > >>>>>>>>>>>> saves
> >> > > > > > >>>>>>>>>>>>>>>>>>> us
> >> > > > > > >>>>>>>>>>>>>>>>>>>           the header
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction
> >> of
> >> > > > always
> >> > > > > > >>>>>>> repartitioning
> >> > > > > > >>>>>>>>>>>> back is
> >> > > > > > >>>>>>>>>>>>>>>>>>>           just not so
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work
> has
> >> > been
> >> > > > > done
> >> > > > > > >>>>> before
> >> > > > > > >>>>>>> we
> >> > > > > > >>>>>>>>>>>> partition
> >> > > > > > >>>>>>>>>>>>>>>>>>>           back and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else
> >> afterwards
> >> > > is
> >> > > > > > really
> >> > > > > > >>>>>>> common.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49,
> Adam
> >> > > > > Bellemare
> >> > > > > > >>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your
> >> > feedback,
> >> > > I
> >> > > > do
> >> > > > > > >>>>>>> appreciate
> >> > > > > > >>>>>>>>> it!
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing
> >> would be
> >> > > > > > possible,
> >> > > > > > >> it
> >> > > > > > >>>>>>> would
> >> > > > > > >>>>>>>>>>>> require
> >> > > > > > >>>>>>>>>>>>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > deserialize
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
> >> implies
> >> > a
> >> > > > > > runtime
> >> > > > > > >>>>>>> overhead.
> >> > > > > > >>>>>>>>> I
> >> > > > > > >>>>>>>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > no
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
> >> avoid
> >> > > the
> >> > > > > > >>>>> overhead.
> >> > > > > > >>>>>>> If
> >> > > > > > >>>>>>>>> this
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> becomes a
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > problem in
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
> >> still
> >> > add
> >> > > > > name
> >> > > > > > >>>>> spacing
> >> > > > > > >>>>>>>>> later
> >> > > > > > >>>>>>>>>>>> on.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go
> with
> >> > > using a
> >> > > > > > >> reserved
> >> > > > > > >>>>>>> string
> >> > > > > > >>>>>>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           document it.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about
> >> the
> >> > > > design
> >> > > > > it
> >> > > > > > >> the
> >> > > > > > >>>>>>> type of
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > If
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the
> proposal
> >> > > > > correctly,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you
> >> have
> >> > > > table1
> >> > > > > > and
> >> > > > > > >>>>> table2
> >> > > > > > >>>>>>>>>>>> swapped.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           Here is how it
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the
> >> records
> >> > > that
> >> > > > > > contain
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>>>>> foreign key
> >> > > > > > >>>>>>>>>>>>>>>>>>>           within their
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> >> > > > > > <a,(fk=A,bar=1)>,
> >> > > > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream:
> >> <A,X>,
> >> > > > <B,Y>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is
> >> > required
> >> > > > to
> >> > > > > > >> extract
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>>>>> foreign
> >> > > > > > >>>>>>>>>>>>>>>>>>> key.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key
> >> mapper:
> >> > (
> >> > > > > value
> >> > > > > > =>
> >> > > > > > >>>>>>> value.fk
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied
> to
> >> > each
> >> > > > > > element
> >> > > > > > >> in
> >> > > > > > >>>>>>>>> table1,
> >> > > > > > >>>>>>>>>>>> and a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           new combined
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> >> > > > > (fk=A,bar=1)>,
> >> > > > > > >>>>> <A-b,
> >> > > > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           <B-c,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events
> >> are
> >> > > > > > >> copartitioned
> >> > > > > > >>>>>>> with
> >> > > > > > >>>>>>>>>>>> table2:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
> >> > > Partition
> >> > > > > 0:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
> >> <A-a,
> >> > > > > > >>>>> (fk=A,bar=1)>,
> >> > > > > > >>>>>>> <A-b,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
> >> > > Partition
> >> > > > > 1:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
> >> <B-c,
> >> > > > > > >> (fk=B,bar=3)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can
> >> be
> >> > > > joined
> >> > > > > > >>>>> together
> >> > > > > > >>>>>>>>> locally
> >> > > > > > >>>>>>>>>>>> by
> >> > > > > > >>>>>>>>>>>>>>>>>>>           applying the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's
> >> design
> >> > > and
> >> > > > > my
> >> > > > > > >>>>> design
> >> > > > > > >>>>>>>>>>>> deviate. My
> >> > > > > > >>>>>>>>>>>>>>>>>>>           design goes
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > on
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
> >> > > post-join
> >> > > > > and
> >> > > > > > >>>>> resolve
> >> > > > > > >>>>>>>>>>>> out-of-order
> >> > > > > > >>>>>>>>>>>>>>>>>>>           arrival of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the
> >> data
> >> > > > keyed
> >> > > > > > just
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>>>>>>> original key.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           I do not
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of
> >> the
> >> > > > > internals
> >> > > > > > >>>>>>> outside of
> >> > > > > > >>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does
> make
> >> > for
> >> > > > > larger
> >> > > > > > >>>>>>> footprint,
> >> > > > > > >>>>>>>>>>>> but it
> >> > > > > > >>>>>>>>>>>>>>>>>>>           removes all
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving
> >> out-of-order
> >> > > > > arrivals
> >> > > > > > >> and
> >> > > > > > >>>>>>>>> handling
> >> > > > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that
> >> this
> >> > > makes
> >> > > > > the
> >> > > > > > >>>>>>> function
> >> > > > > > >>>>>>>>> much
> >> > > > > > >>>>>>>>>>>>>>>>>>> easier
> >> > > > > > >>>>>>>>>>>>>>>>>>>           to use.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this
> >> helps
> >> > > > resolve
> >> > > > > > your
> >> > > > > > >>>>>>>>> questions,
> >> > > > > > >>>>>>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           please feel
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on
> >> your
> >> > > mind.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at
> >> 8:36
> >> > > PM,
> >> > > > > > >>>>> Matthias J.
> >> > > > > > >>>>>>>>> Sax <
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io
> >> > > <mailto:
> >> > > > > > >>>>>>>>>>>> matthias@confluent.io>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up
> >> on
> >> > > this
> >> > > > > > >> thread. I
> >> > > > > > >>>>>>> did
> >> > > > > > >>>>>>>>> not
> >> > > > > > >>>>>>>>>>>> read
> >> > > > > > >>>>>>>>>>>>>>>>>>>           everything so
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share
> >> couple
> >> > of
> >> > > > > > initial
> >> > > > > > >>>>>>> thoughts:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think
> there
> >> is
> >> > a
> >> > > > > > >> fundamental
> >> > > > > > >>>>>>>>>>>> difference
> >> > > > > > >>>>>>>>>>>>>>>>>>>           between header
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and
> KP-258.
> >> > For
> >> > > > 258,
> >> > > > > > we
> >> > > > > > >> add
> >> > > > > > >>>>>>>>> headers
> >> > > > > > >>>>>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka
> >> Streams
> >> > > and
> >> > > > > > nobody
> >> > > > > > >>>>>>> else is
> >> > > > > > >>>>>>>>>>>> supposed
> >> > > > > > >>>>>>>>>>>>>>>>>>>           to write
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > into
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no
> user
> >> > > header
> >> > > > > are
> >> > > > > > >>>>> written
> >> > > > > > >>>>>>> into
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
> >> > > conflicts.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't
> >> see
> >> > a
> >> > > > big
> >> > > > > > issue
> >> > > > > > >>>>> with
> >> > > > > > >>>>>>>>> using
> >> > > > > > >>>>>>>>>>>>>>>>>>>           headers within
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we
> document
> >> it,
> >> > > we
> >> > > > > can
> >> > > > > > >> have
> >> > > > > > >>>>>>> some
> >> > > > > > >>>>>>>>>>>> "reserved"
> >> > > > > > >>>>>>>>>>>>>>>>>>>           header keys
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed
> >> to
> >> > use
> >> > > > > when
> >> > > > > > >>>>>>> processing
> >> > > > > > >>>>>>>>>>>> data with
> >> > > > > > >>>>>>>>>>>>>>>>>>>           Kafka
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > Streams.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be
> >> ok.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a
> safe
> >> > way
> >> > > to
> >> > > > > > avoid
> >> > > > > > >>>>>>>>> conflicts,
> >> > > > > > >>>>>>>>>>>> since
> >> > > > > > >>>>>>>>>>>>>>>>>>> these
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > headers
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in
> >> internal
> >> > > > topics
> >> > > > > (I
> >> > > > > > >>>>> think):
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and
> >> > changelog
> >> > > > > > topics,
> >> > > > > > >> we
> >> > > > > > >>>>> can
> >> > > > > > >>>>>>>>>>>> namespace
> >> > > > > > >>>>>>>>>>>>>>>>>>>           all headers:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined
> headers
> >> > are
> >> > > > > > >> namespaced
> >> > > > > > >>>>> as
> >> > > > > > >>>>>>>>>>>> "external."
> >> > > > > > >>>>>>>>>>>>>>>>>>> +
> >> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers
> are
> >> > > > > > namespaced as
> >> > > > > > >>>>>>>>>>>> "internal." +
> >> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing
> >> would
> >> > be
> >> > > > > > >> possible,
> >> > > > > > >>>>> it
> >> > > > > > >>>>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>>>> require
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
> >> implies
> >> > a
> >> > > > > > runtime
> >> > > > > > >>>>>>> overhead.
> >> > > > > > >>>>>>>>> I
> >> > > > > > >>>>>>>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > no
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
> >> avoid
> >> > > the
> >> > > > > > >>>>> overhead.
> >> > > > > > >>>>>>> If
> >> > > > > > >>>>>>>>> this
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> becomes a
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > problem in
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
> >> still
> >> > add
> >> > > > > name
> >> > > > > > >>>>> spacing
> >> > > > > > >>>>>>>>> later
> >> > > > > > >>>>>>>>>>>> on.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about
> >> the
> >> > > > design
> >> > > > > > it
> >> > > > > > >> the
> >> > > > > > >>>>>>> type
> >> > > > > > >>>>>>>>> of
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the
> proposal
> >> > > > > correctly,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1
> =
> >> ...
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2
> =
> >> ...
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3>
> >> joinedTable
> >> > =
> >> > > > > > >>>>>>>>>>>> table1.join(table2,...);
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
> >> > > `joinedTable`
> >> > > > > has
> >> > > > > > >> the
> >> > > > > > >>>>>>> same
> >> > > > > > >>>>>>>>> key
> >> > > > > > >>>>>>>>>>>> as the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           left input
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not
> >> work
> >> > > > because
> >> > > > > > if
> >> > > > > > >>>>> table2
> >> > > > > > >>>>>>>>>>>> contains
> >> > > > > > >>>>>>>>>>>>>>>>>>>           multiple rows
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in
> >> > table1
> >> > > > > > (what is
> >> > > > > > >>>>> the
> >> > > > > > >>>>>>> main
> >> > > > > > >>>>>>>>>>>> purpose
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > foreign
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result
> table
> >> > > would
> >> > > > > only
> >> > > > > > >>>>>>> contain a
> >> > > > > > >>>>>>>>>>>> single
> >> > > > > > >>>>>>>>>>>>>>>>>>>           join result,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > but
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream:
> >> <A,X>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
> >> > > > <a,(A,1)>,
> >> > > > > > >>>>> <b,(A,2)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a
> >> > > foreign
> >> > > > > key
> >> > > > > > to
> >> > > > > > >>>>>>> table1
> >> > > > > > >>>>>>>>> key
> >> > > > > > >>>>>>>>>>>> (ie,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > If
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the
> same
> >> key
> >> > > as
> >> > > > > key
> >> > > > > > of
> >> > > > > > >>>>>>> table1,
> >> > > > > > >>>>>>>>> this
> >> > > > > > >>>>>>>>>>>>>>>>>>>           implies that the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be
> >> <A,
> >> > > > > > join(X,1)>
> >> > > > > > >> or
> >> > > > > > >>>>> <A,
> >> > > > > > >>>>>>>>>>>> join(X,2)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           but not
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > both.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the
> >> same
> >> > > > key,
> >> > > > > > >>>>> whatever
> >> > > > > > >>>>>>>>> result
> >> > > > > > >>>>>>>>>>>> record
> >> > > > > > >>>>>>>>>>>>>>>>>>>           we emit
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > later,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the
> previous
> >> > > result.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason
> why
> >> Jan
> >> > > > > > originally
> >> > > > > > >>>>>>> proposed
> >> > > > > > >>>>>>>>>>>> to use
> >> > > > > > >>>>>>>>>>>>>>>>>>> a
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > combination
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of
> >> the
> >> > > input
> >> > > > > > tables
> >> > > > > > >>>>> as
> >> > > > > > >>>>>>> key
> >> > > > > > >>>>>>>>> of
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           output table.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the
> >> > output
> >> > > > > table
> >> > > > > > >>>>> unique
> >> > > > > > >>>>>>> and
> >> > > > > > >>>>>>>>> we
> >> > > > > > >>>>>>>>>>>> can
> >> > > > > > >>>>>>>>>>>>>>>>>>>           store both in
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> >> > > > > join(X,1)>,
> >> > > > > > >> <A-b,
> >> > > > > > >>>>>>>>>>>> join(X,2)>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM,
> Jan
> >> > > > Filipiak
> >> > > > > > >> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark
> >> could
> >> > be
> >> > > > > > >>>>> disregarded.
> >> > > > > > >>>>>>> The
> >> > > > > > >>>>>>>>>>>> decision
> >> > > > > > >>>>>>>>>>>>>>>>>>>           about the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size
> of
> >> > the
> >> > > > > > >> aggregated
> >> > > > > > >>>>>>> map.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long
> >> maps
> >> > > > would
> >> > > > > be
> >> > > > > > >>>>>>> unpacked
> >> > > > > > >>>>>>>>> and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > element
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published
> as
> >> > > delete.
> >> > > > > Any
> >> > > > > > >>>>> other
> >> > > > > > >>>>>>> count
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in
> >> > > "waiting
> >> > > > > for
> >> > > > > > >>>>> correct
> >> > > > > > >>>>>>>>>>>> deletes to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29,
> >> Adam
> >> > > > > > Bellemare
> >> > > > > > >>>>>>> wrote:
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I
> >> could
> >> > > > > replace
> >> > > > > > >> the
> >> > > > > > >>>>>>> second
> >> > > > > > >>>>>>>>>>>>>>>>>>>           repartition store
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > and
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store
> with
> >> a
> >> > > > groupBy
> >> > > > > > and
> >> > > > > > >>>>>>> reduce.
> >> > > > > > >>>>>>>>>>>> However,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           it looks
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > like
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store
> >> the
> >> > > > > > highwater
> >> > > > > > >>>>> value
> >> > > > > > >>>>>>>>> within
> >> > > > > > >>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           materialized
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival
> of
> >> > > > > > out-of-order
> >> > > > > > >>>>>>> records
> >> > > > > > >>>>>>>>>>>> (assuming
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>> my
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...).
> >> This
> >> > > in
> >> > > > > > effect
> >> > > > > > >> is
> >> > > > > > >>>>>>> the
> >> > > > > > >>>>>>>>> same
> >> > > > > > >>>>>>>>>>>> as
> >> > > > > > >>>>>>>>>>>>>>>>>>> the
> >> > > > > > >>>>>>>>>>>>>>>>>>>           design I
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two
> >> tables
> >> > > > merged
> >> > > > > > >>>>> together.
> >> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
> >> > > > > > >>>>>>>>>>>>>>>>>>>           >
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>> --
> >> > > > > > >>>>>>>>>>>>>>>>> -- Guozhang
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>> --
> >> > > > > > >>>>>>>>>>>>>>>> -- Guozhang
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>>
> >> > > > > > >>>>>>>>>>>
> >> > > > > > >>>>>>>>>>
> >> > > > > > >>>>>>>>>
> >> > > > > > >>>>>>>>
> >> > > > > > >>>>>>>
> >> > > > > > >>>>>>
> >> > > > > > >>>>>
> >> > > > > > >>>>
> >> > > > > > >>>
> >> > > > > > >>
> >> > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >>
> >
>
> --
> -- Guozhang
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

Just made a pass on your diagram (nice hand-drawing btw!), and obviously we
are thinking about the same thing :) A neat difference that I like, is that
in the pre-join repartition topic we can still send message in the format
of `K=k, V=(i=2)` while using "i" as the partition key in StreamsPartition,
this way we do not need to even augment the key for the repartition topic,
but just do a projection on the foreign key part but trim all other fields:
as long as we still materialize the store as `A-2` co-located with the
right KTable, that is fine.

As I mentioned in my previous email, I also think this has a few advantages
on saving over-the-wire bytes as well as disk bytes.

Guozhang


On Mon, Dec 17, 2018 at 3:17 PM John Roesler <jo...@confluent.io> wrote:

> Hi Guozhang,
>
> Thanks for taking a look! I think Adam's already addressed your questions
> as well as I could have.
>
> Hi Adam,
>
> Thanks for updating the KIP. It looks great, especially how all the
> need-to-know information is right at the top, followed by the details.
>
> Also, thanks for that high-level diagram. Actually, now that I'm looking
> at it, I think part of my proposal got lost in translation, although I do
> think that what you have there is also correct.
>
> I sketched up a crude diagram based on yours and attached it to the KIP
> (I'm not sure if attached or inline images work on the mailing list):
> https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
> . It's also attached to this email for convenience.
>
> Hopefully, you can see how it's intended to line up, and which parts are
> modified.
> At a high level, instead of performing the join on the right-hand side,
> we're essentially just registering interest, like "LHS key A wishes to
> receive updates for RHS key 2". Then, when there is a new "interest" or any
> updates to the RHS records, it "broadcasts" its state back to the LHS
> records who are interested in it.
>
> Thus, instead of sending the LHS values to the RHS joiner workers and then
> sending the join results back to the LHS worke be co-partitioned and
> validated, we instead only send the LHS *keys* to the RHS workers and then
> only the RHS k/v back to be joined by the LHS worker.
>
> I've been considering both your diagram and mine, and I *think* what I'm
> proposing has a few advantages.
>
> Here are some points of interest as you look at the diagram:
> * When we extract the foreign key and send it to the Pre-Join Repartition
> Topic, we can send only the FK/PK pair. There's no need to worry about
> custom partitioner logic, since we can just use the foreign key plainly as
> the repartition record key. Also, we save on transmitting the LHS value,
> since we only send its key in this step.
> * We also only need to store the RHSKey:LHSKey mapping in the
> MaterializedSubscriptionStore, saving on disk. We can use the same rocks
> key format you proposed and the same algorithm involving range scans when
> the RHS records get updated.
> * Instead of joining on the right side, all we do is compose a
> re-repartition record so we can broadcast the RHS k/v pair back to the
> original LHS partition. (this is what the "rekey" node is doing)
> * Then, there is a special kind of Joiner that's co-resident in the same
> StreamTask as the LHS table, subscribed to the Post-Join Repartition Topic.
> ** This Joiner is *not* triggered directly by any changes in the LHS
> KTable. Instead, LHS events indirectly trigger the join via the whole
> lifecycle.
> ** For each event arriving from the Post-Join Repartition Topic, the
> Joiner looks up the corresponding record in the LHS KTable. It validates
> the FK as you noted, discarding any inconsistent events. Otherwise, it
> unpacks the RHS K/V pair and invokes the ValueJoiner to obtain the join
> result
> ** Note that the Joiner itself is stateless, so materializing the join
> result is optional, just as with the 1:1 joins.
>
> So in summary:
> * instead of transmitting the LHS keys and values to the right and the
> JoinResult back to the left, we only transmit the LHS keys to the right and
> the RHS values to the left. Assuming the average RHS value is on smaller
> than or equal to the average join result size, it's a clear win on broker
> traffic. I think this is actually a reasonable assumption, which we can
> discuss more if you're suspicious.
> * we only need one copy of the data (the left and right tables need to be
> materialized) and one extra copy of the PK:FK pairs in the Materialized
> Subscription Store. Materializing the join result is optional, just as with
> the existing 1:1 joins.
> * we still need the fancy range-scan algorithm on the right to locate all
> interested LHS keys when a RHS value is updated, but we don't need a custom
> partitioner for either repartition topic (this is of course a modification
> we could make to your version as well)
>
> How does this sound to you? (And did I miss anything?)
> -John
>
> On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi John & Guozhang
>>
>> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the KIP,
>> pruned much of what I wrote and put a simplified diagram near the top to
>> illustrate the workflow. I encapsulated Jan's content at the bottom of the
>> document. I believe it is simpler to read by far now.
>>
>> @Guozhang Wang <wa...@gmail.com>:
>> > #1: rekey left table
>> >   -> source from the left upstream, send to rekey-processor to generate
>> combined key, and then sink to copartition topic.
>> Correct.
>>
>> > #2: first-join with right table
>> >   -> source from the right table upstream, materialize the right table.
>> >   -> source from the co-partition topic, materialize the rekeyed left
>> table, join with the right table, rekey back, and then sink to the
>> rekeyed-back topic.
>> Almost - I cleared up the KIP. We do not rekey back yet, as I need the
>> Foreign-Key value generated in #1 above to compare in the resolution
>> stage.
>>
>> > #3: second join
>> >    -> source from the rekeyed-back topic, materialize the rekeyed back
>> table.
>> >   -> source from the left upstream, materialize the left table, join
>> with
>> the rekeyed back table.
>> Almost - As each event comes in, we just run it through a stateful
>> processor that checks the original ("This") KTable for the key. The value
>> payload then has the foreignKeyExtractor applied again as in Part #1
>> above,
>> and gets the current foreign key. Then we compare it to the joined event
>> that we are currently resolving. If they have the same foreign-key,
>> propagate the result out. If they don't, throw the event away.
>>
>> The end result is that we do need to materialize 2 additional tables
>> (left/this-combinedkey table, and the final Joined table) as I've
>> illustrated in the updated KIP. I hope the diagram clears it up a lot
>> better. Please let me know.
>>
>> Thanks again
>> Adam
>>
>>
>>
>>
>> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>> > John,
>> >
>> > Thanks a lot for the suggestions on refactoring the wiki, I agree with
>> you
>> > that we should consider the KIP proposal to be easily understood by
>> anyone
>> > in the future to read, and hence should provide a good summary on the
>> > user-facing interfaces, as well as rejected alternatives to represent
>> > briefly "how we came a long way to this conclusion, and what we have
>> > argued, disagreed, and agreed about, etc" so that readers do not need to
>> > dig into the DISCUSS thread to get all the details. We can, of course,
>> keep
>> > the implementation details like "workflows" on the wiki page as a
>> addendum
>> > section since it also has correlations.
>> >
>> > Regarding your proposal on comment 6): that's a very interesting idea!
>> Just
>> > to clarify that I understands it fully correctly: the proposal's
>> resulted
>> > topology is still the same as the current proposal, where we will have 3
>> > sub-topologies for this operator:
>> >
>> > #1: rekey left table
>> >    -> source from the left upstream, send to rekey-processor to generate
>> > combined key, and then sink to copartition topic.
>> >
>> > #2: first-join with right table
>> >    -> source from the right table upstream, materialize the right table.
>> >    -> source from the co-partition topic, materialize the rekeyed left
>> > table, join with the right table, rekey back, and then sink to the
>> > rekeyed-back topic.
>> >
>> > #3: second join
>> >    -> source from the rekeyed-back topic, materialize the rekeyed back
>> > table.
>> >    -> source from the left upstream, materialize the left table, join
>> with
>> > the rekeyed back table.
>> >
>> > Sub-topology #1 and #3 may be merged to a single sub-topology since
>> both of
>> > them read from the left table source stream. In this workflow, we need
>> to
>> > materialize 4 tables (left table in #3, right table in #2, rekeyed left
>> > table in #2, rekeyed-back table in #3), and 2 repartition topics
>> > (copartition topic, rekeyed-back topic).
>> >
>> > Compared with Adam's current proposal in the workflow overview, it has
>> the
>> > same num.materialize tables (left table, rekeyed left table, right
>> table,
>> > out-of-ordering resolver table), and same num.internal topics (two). The
>> > advantage is that on the copartition topic, we can save bandwidth by not
>> > sending value, and in #2 the rekeyed left table is smaller since we do
>> not
>> > have any values to materialize. Is that right?
>> >
>> >
>> > Guozhang
>> >
>> >
>> >
>> > On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io> wrote:
>> >
>> > > Hi Adam,
>> > >
>> > > Given that the committers are all pretty busy right now, I think that
>> it
>> > > would help if you were to refactor the KIP a little to reduce the
>> > workload
>> > > for reviewers.
>> > >
>> > > I'd recommend the following changes:
>> > > * relocate all internal details to a section at the end called
>> something
>> > > like "Implementation Notes" or something like that.
>> > > * rewrite the rest of the KIP to be a succinct as possible and mention
>> > only
>> > > publicly-facing API changes.
>> > > ** for example, the interface that you've already listed there, as
>> well
>> > as
>> > > a textual description of the guarantees we'll be providing (join
>> result
>> > is
>> > > copartitioned with the LHS, and the join result is guaranteed correct)
>> > >
>> > > A good target would be that the whole main body of the KIP, including
>> > > Status, Motivation, Proposal, Justification, and Rejected Alternatives
>> > all
>> > > fit "above the fold" (i.e., all fit on the screen at a comfortable
>> zoom
>> > > level).
>> > > I think the only real Rejected Alternative that bears mention at this
>> > point
>> > > is KScatteredTable, which you could just include the executive
>> summary on
>> > > (no implementation details), and link to extra details in the
>> > > Implementation Notes section.
>> > >
>> > > Taking a look at the wiki page, ~90% of the text there is internal
>> > detail,
>> > > which is useful for the dubious, but doesn't need to be ratified in a
>> > vote
>> > > (and would be subject to change without notice in the future anyway).
>> > > There's also a lot of conflicting discussion, as you've very
>> respectfully
>> > > tried to preserve the original proposal from Jan while adding your
>> own.
>> > > Isolating all this information in a dedicated section at the bottom
>> frees
>> > > the voters up to focus on the public API part of the proposal, which
>> is
>> > > really all they need to consider.
>> > >
>> > > Plus, it'll be clear to future readers which parts of the document are
>> > > enduring, and which parts are a snapshot of our implementation
>> thinking
>> > at
>> > > the time.
>> > >
>> > > I'm suggesting this because I suspect that the others haven't made
>> time
>> > to
>> > > review it partly because it seems daunting. If it seems like it would
>> be
>> > a
>> > > huge time investment to review, people will just keep putting it off.
>> But
>> > > if the KIP is a single page, then they'll be more inclined to give it
>> a
>> > > read.
>> > >
>> > > Honestly, I don't think the KIP itself is that controversial (apart
>> from
>> > > the scattered table thing (sorry, Jan) ). Most of the discussion has
>> been
>> > > around the implementation, which we can continue more effectively in
>> a PR
>> > > once the KIP has passed.
>> > >
>> > > How does that sound?
>> > > -John
>> > >
>> > > On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
>> adam.bellemare@gmail.com
>> > >
>> > > wrote:
>> > >
>> > > > 1) I believe that the resolution mechanism John has proposed is
>> > > sufficient
>> > > > - it is clean and easy and doesn't require additional RocksDB
>> stores,
>> > > which
>> > > > reduces the footprint greatly. I don't think we need to resolve
>> based
>> > on
>> > > > timestamp or offset anymore, but if we decide to do to that would be
>> > > within
>> > > > the bounds of the existing API.
>> > > >
>> > > > 2) Is the current API sufficient, or does it need to be altered to
>> go
>> > > back
>> > > > to vote?
>> > > >
>> > > > 3) KScatteredTable implementation can always be added in a future
>> > > revision.
>> > > > This API does not rule it out. This implementation of this function
>> > would
>> > > > simply be replaced with `KScatteredTable.resolve()` while still
>> > > maintaining
>> > > > the existing API, thereby giving both features as Jan outlined
>> earlier.
>> > > > Would this work?
>> > > >
>> > > >
>> > > > Thanks Guozhang, John and Jan
>> > > >
>> > > >
>> > > >
>> > > >
>> > > > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io>
>> > wrote:
>> > > >
>> > > > > Hi, all,
>> > > > >
>> > > > > >> In fact, we
>> > > > > >> can just keep a single final-result store with timestamps and
>> > reject
>> > > > > values
>> > > > > >> that have a smaller timestamp, is that right?
>> > > > >
>> > > > > > Which is the correct output should at least be decided on the
>> > offset
>> > > of
>> > > > > > the original message.
>> > > > >
>> > > > > Thanks for this point, Jan.
>> > > > >
>> > > > > KIP-258 is merely to allow embedding the record timestamp  in the
>> k/v
>> > > > > store,
>> > > > > as well as providing a storage-format upgrade path.
>> > > > >
>> > > > > I might have missed it, but I think we have yet to discuss whether
>> > it's
>> > > > > safe
>> > > > > or desirable just to swap topic-ordering our for
>> timestamp-ordering.
>> > > This
>> > > > > is
>> > > > > a very deep topic, and I think it would only pollute the current
>> > > > > discussion.
>> > > > >
>> > > > > What Adam has proposed is safe, given the *current* ordering
>> > semantics
>> > > > > of the system. If we can agree on his proposal, I think we can
>> merge
>> > > the
>> > > > > feature well before the conversation about timestamp ordering even
>> > > takes
>> > > > > place, much less reaches a conclusion. In the mean time, it would
>> > seem
>> > > to
>> > > > > be unfortunate to have one join operator with different ordering
>> > > > semantics
>> > > > > from every other KTable operator.
>> > > > >
>> > > > > If and when that timestamp discussion takes place, many (all?)
>> KTable
>> > > > > operations
>> > > > > will need to be updated, rendering the many:one join a small
>> marginal
>> > > > cost.
>> > > > >
>> > > > > And, just to plug it again, I proposed an algorithm above that I
>> > > believe
>> > > > > provides
>> > > > > correct ordering without any additional metadata, and regardless
>> of
>> > the
>> > > > > ordering semantics. I didn't bring it up further, because I felt
>> the
>> > > KIP
>> > > > > only needs
>> > > > > to agree on the public API, and we can discuss the implementation
>> at
>> > > > > leisure in
>> > > > > a PR...
>> > > > >
>> > > > > Thanks,
>> > > > > -John
>> > > > >
>> > > > >
>> > > > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
>> > Jan.Filipiak@trivago.com
>> > > >
>> > > > > wrote:
>> > > > >
>> > > > > >
>> > > > > >
>> > > > > > On 10.12.2018 07:42, Guozhang Wang wrote:
>> > > > > > > Hello Adam / Jan / John,
>> > > > > > >
>> > > > > > > Sorry for being late on this thread! I've finally got some
>> time
>> > > this
>> > > > > > > weekend to cleanup a load of tasks on my queue (actually I've
>> > also
>> > > > > > realized
>> > > > > > > there are a bunch of other things I need to enqueue while
>> > cleaning
>> > > > them
>> > > > > > up
>> > > > > > > --- sth I need to improve on my side). So here are my
>> thoughts:
>> > > > > > >
>> > > > > > > Regarding the APIs: I like the current written API in the KIP.
>> > More
>> > > > > > > generally I'd prefer to keep the 1) one-to-many join
>> > > functionalities
>> > > > as
>> > > > > > > well as 2) other join types than inner as separate KIPs since
>> 1)
>> > > may
>> > > > > > worth
>> > > > > > > a general API refactoring that can benefit not only foreignkey
>> > > joins
>> > > > > but
>> > > > > > > collocate joins as well (e.g. an extended proposal of
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
>> > > > > > ),
>> > > > > > > and I'm not sure if other join types would actually be needed
>> > > (maybe
>> > > > > left
>> > > > > > > join still makes sense), so it's better to
>> > > > > wait-for-people-to-ask-and-add
>> > > > > > > than add-sth-that-no-one-uses.
>> > > > > > >
>> > > > > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
>> > > > > > > KScatteredTable for users to inject their own optimization:
>> I'd
>> > > > prefer
>> > > > > to
>> > > > > > > do the current option as-is, and my main rationale is for
>> > > > optimization
>> > > > > > > rooms inside the Streams internals and the API succinctness.
>> For
>> > > > > advanced
>> > > > > > > users who may indeed prefer KScatteredTable and do their own
>> > > > > > optimization,
>> > > > > > > while it is too much of the work to use Processor API
>> directly, I
>> > > > think
>> > > > > > we
>> > > > > > > can still extend the current API to support it in the future
>> if
>> > it
>> > > > > > becomes
>> > > > > > > necessary.
>> > > > > >
>> > > > > > no internal optimization potential. it's a myth
>> > > > > >
>> > > > > > ¯\_(ツ)_/¯
>> > > > > >
>> > > > > > :-)
>> > > > > >
>> > > > > > >
>> > > > > > > Another note about step 4) resolving out-of-ordering data, as
>> I
>> > > > > mentioned
>> > > > > > > before I think with KIP-258 (embedded timestamp with key-value
>> > > store)
>> > > > > we
>> > > > > > > can actually make this step simpler than the current
>> proposal. In
>> > > > fact,
>> > > > > > we
>> > > > > > > can just keep a single final-result store with timestamps and
>> > > reject
>> > > > > > values
>> > > > > > > that have a smaller timestamp, is that right?
>> > > > > >
>> > > > > > Which is the correct output should at least be decided on the
>> > offset
>> > > of
>> > > > > > the original message.
>> > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > That's all I have in mind now. Again, great appreciation to
>> Adam
>> > to
>> > > > > make
>> > > > > > > such HUGE progress on this KIP!
>> > > > > > >
>> > > > > > >
>> > > > > > > Guozhang
>> > > > > > >
>> > > > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
>> > > > Jan.Filipiak@trivago.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > >> If they don't find the time:
>> > > > > > >> They usually take the opposite path from me :D
>> > > > > > >> so the answer would be clear.
>> > > > > > >>
>> > > > > > >> hence my suggestion to vote.
>> > > > > > >>
>> > > > > > >>
>> > > > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
>> > > > > > >>> Hi Guozhang and Matthias
>> > > > > > >>>
>> > > > > > >>> I know both of you are quite busy, but we've gotten this KIP
>> > to a
>> > > > > point
>> > > > > > >>> where we need more guidance on the API (perhaps a bit of a
>> > > > > tie-breaker,
>> > > > > > >> if
>> > > > > > >>> you will). If you have anyone else you may think should
>> look at
>> > > > this,
>> > > > > > >>> please tag them accordingly.
>> > > > > > >>>
>> > > > > > >>> The scenario is as such:
>> > > > > > >>>
>> > > > > > >>> Current Option:
>> > > > > > >>> API:
>> > > > > > >>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>> > > > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
>> > > partition
>> > > > > with
>> > > > > > >> the
>> > > > > > >>> foreignKey (repartition 1)
>> > > > > > >>> 2) Join the data
>> > > > > > >>> 3) Shuffle the data back to the original node (repartition
>> 2)
>> > > > > > >>> 4) Resolve out-of-order arrival / race condition due to
>> > > foreign-key
>> > > > > > >> changes.
>> > > > > > >>>
>> > > > > > >>> Alternate Option:
>> > > > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
>> > > > > > >>> - It would be keyed on a wrapped key function:
>> <CombinedKey<KO,
>> > > K>,
>> > > > > VR>
>> > > > > > >> (KO
>> > > > > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
>> > > > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
>> > > otherwise a
>> > > > > > user
>> > > > > > >>> would be able to perform additional functions directly from
>> the
>> > > > > > >>> KScatteredTable (TBD - currently out of scope).
>> > > > > > >>> - John's analysis 2-emails up is accurate as to the
>> tradeoffs.
>> > > > > > >>>
>> > > > > > >>> Current Option is coded as-is. Alternate option is possible,
>> > but
>> > > > will
>> > > > > > >>> require for implementation details to be made in the API and
>> > some
>> > > > > > >> exposure
>> > > > > > >>> of new data structures into the API (ie: CombinedKey).
>> > > > > > >>>
>> > > > > > >>> I appreciate any insight into this.
>> > > > > > >>>
>> > > > > > >>> Thanks.
>> > > > > > >>>
>> > > > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
>> > > > > > adam.bellemare@gmail.com>
>> > > > > > >>> wrote:
>> > > > > > >>>
>> > > > > > >>>> Hi John
>> > > > > > >>>>
>> > > > > > >>>> Thanks for your feedback and assistance. I think your
>> summary
>> > is
>> > > > > > >> accurate
>> > > > > > >>>> from my perspective. Additionally, I would like to add that
>> > > there
>> > > > > is a
>> > > > > > >> risk
>> > > > > > >>>> of inconsistent final states without performing the
>> > resolution.
>> > > > This
>> > > > > > is
>> > > > > > >> a
>> > > > > > >>>> major concern for me as most of the data I have dealt with
>> is
>> > > > > produced
>> > > > > > >> by
>> > > > > > >>>> relational databases. We have seen a number of cases where
>> a
>> > > user
>> > > > in
>> > > > > > the
>> > > > > > >>>> Rails UI has modified the field (foreign key), realized
>> they
>> > > made
>> > > > a
>> > > > > > >>>> mistake, and then updated the field again with a new key.
>> The
>> > > > events
>> > > > > > are
>> > > > > > >>>> propagated out as they are produced, and as such we have
>> had
>> > > > > > real-world
>> > > > > > >>>> cases where these inconsistencies were propagated
>> downstream
>> > as
>> > > > the
>> > > > > > >> final
>> > > > > > >>>> values due to the race conditions in the fanout of the
>> data.
>> > > > > > >>>>
>> > > > > > >>>> This solution that I propose values correctness of the
>> final
>> > > > result
>> > > > > > over
>> > > > > > >>>> other factors.
>> > > > > > >>>>
>> > > > > > >>>> We could always move this function over to using a
>> > > KScatteredTable
>> > > > > > >>>> implementation in the future, and simply deprecate it this
>> > join
>> > > > API
>> > > > > in
>> > > > > > >>>> time. I think I would like to hear more from some of the
>> other
>> > > > major
>> > > > > > >>>> committers on which course of action they would think is
>> best
>> > > > before
>> > > > > > any
>> > > > > > >>>> more coding is done.
>> > > > > > >>>>
>> > > > > > >>>> Thanks again
>> > > > > > >>>>
>> > > > > > >>>> Adam
>> > > > > > >>>>
>> > > > > > >>>>
>> > > > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
>> > john@confluent.io>
>> > > > > > wrote:
>> > > > > > >>>>
>> > > > > > >>>>> Hi Jan and Adam,
>> > > > > > >>>>>
>> > > > > > >>>>> Wow, thanks for doing that test, Adam. Those results are
>> > > > > encouraging.
>> > > > > > >>>>>
>> > > > > > >>>>> Thanks for your performance experience as well, Jan. I
>> agree
>> > > that
>> > > > > > >> avoiding
>> > > > > > >>>>> unnecessary join outputs is especially important when the
>> > > fan-out
>> > > > > is
>> > > > > > so
>> > > > > > >>>>> high. I suppose this could also be built into the
>> > > implementation
>> > > > > > we're
>> > > > > > >>>>> discussing, but it wouldn't have to be specified in the
>> KIP
>> > > > (since
>> > > > > > >> it's an
>> > > > > > >>>>> API-transparent optimization).
>> > > > > > >>>>>
>> > > > > > >>>>> As far as whether or not to re-repartition the data, I
>> didn't
>> > > > bring
>> > > > > > it
>> > > > > > >> up
>> > > > > > >>>>> because it sounded like the two of you agreed to leave the
>> > KIP
>> > > > > as-is,
>> > > > > > >>>>> despite the disagreement.
>> > > > > > >>>>>
>> > > > > > >>>>> If you want my opinion, I feel like both approaches are
>> > > > reasonable.
>> > > > > > >>>>> It sounds like Jan values more the potential for
>> developers
>> > to
>> > > > > > optimize
>> > > > > > >>>>> their topologies to re-use the intermediate nodes, whereas
>> > Adam
>> > > > > > places
>> > > > > > >>>>> more
>> > > > > > >>>>> value on having a single operator that people can use
>> without
>> > > > extra
>> > > > > > >> steps
>> > > > > > >>>>> at the end.
>> > > > > > >>>>>
>> > > > > > >>>>> Personally, although I do find it exceptionally annoying
>> > when a
>> > > > > > >> framework
>> > > > > > >>>>> gets in my way when I'm trying to optimize something, it
>> > seems
>> > > > > better
>> > > > > > >> to
>> > > > > > >>>>> go
>> > > > > > >>>>> for a single operation.
>> > > > > > >>>>> * Encapsulating the internal transitions gives us
>> significant
>> > > > > > latitude
>> > > > > > >> in
>> > > > > > >>>>> the implementation (for example, joining only at the end,
>> not
>> > > in
>> > > > > the
>> > > > > > >>>>> middle
>> > > > > > >>>>> to avoid extra data copying and out-of-order resolution;
>> how
>> > we
>> > > > > > >> represent
>> > > > > > >>>>> the first repartition keys (combined keys vs. value
>> vectors),
>> > > > > etc.).
>> > > > > > >> If we
>> > > > > > >>>>> publish something like a KScatteredTable with the
>> > > > right-partitioned
>> > > > > > >> joined
>> > > > > > >>>>> data, then the API pretty much locks in the
>> implementation as
>> > > > well.
>> > > > > > >>>>> * The API seems simpler to understand and use. I do mean
>> > > "seems";
>> > > > > if
>> > > > > > >>>>> anyone
>> > > > > > >>>>> wants to make the case that KScatteredTable is actually
>> > > simpler,
>> > > > I
>> > > > > > >> think
>> > > > > > >>>>> hypothetical usage code would help. From a relational
>> algebra
>> > > > > > >> perspective,
>> > > > > > >>>>> it seems like KTable.join(KTable) should produce a new
>> KTable
>> > > in
>> > > > > all
>> > > > > > >>>>> cases.
>> > > > > > >>>>> * That said, there might still be room in the API for a
>> > > different
>> > > > > > >>>>> operation
>> > > > > > >>>>> like what Jan has proposed to scatter a KTable, and then
>> do
>> > > > things
>> > > > > > like
>> > > > > > >>>>> join, re-group, etc from there... I'm not sure; I haven't
>> > > thought
>> > > > > > >> through
>> > > > > > >>>>> all the consequences yet.
>> > > > > > >>>>>
>> > > > > > >>>>> This is all just my opinion after thinking over the
>> > discussion
>> > > so
>> > > > > > >> far...
>> > > > > > >>>>> -John
>> > > > > > >>>>>
>> > > > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>> > > > > > >> adam.bellemare@gmail.com>
>> > > > > > >>>>> wrote:
>> > > > > > >>>>>
>> > > > > > >>>>>> Updated the PR to take into account John's feedback.
>> > > > > > >>>>>>
>> > > > > > >>>>>> I did some preliminary testing for the performance of the
>> > > > > > prefixScan.
>> > > > > > >> I
>> > > > > > >>>>>> have attached the file, but I will also include the text
>> in
>> > > the
>> > > > > body
>> > > > > > >>>>> here
>> > > > > > >>>>>> for archival purposes (I am not sure what happens to
>> > attached
>> > > > > > files).
>> > > > > > >> I
>> > > > > > >>>>>> also updated the PR and the KIP accordingly.
>> > > > > > >>>>>>
>> > > > > > >>>>>> Summary: It scales exceptionally well for scanning large
>> > > values
>> > > > of
>> > > > > > >>>>>> records. As Jan mentioned previously, the real issue
>> would
>> > be
>> > > > more
>> > > > > > >>>>> around
>> > > > > > >>>>>> processing the resulting records after obtaining them.
>> For
>> > > > > instance,
>> > > > > > >> it
>> > > > > > >>>>>> takes approximately ~80-120 mS to flush the buffer and a
>> > > further
>> > > > > > >>>>> ~35-85mS
>> > > > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of
>> them.
>> > > > > Iterating
>> > > > > > >>>>>> through the records just to generate a simple count
>> takes ~
>> > 40
>> > > > > times
>> > > > > > >>>>> longer
>> > > > > > >>>>>> than the flush + scan combined.
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> ============================================================================================
>> > > > > > >>>>>> Setup:
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> ============================================================================================
>> > > > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
>> > > (Xmx512m,
>> > > > > > >> Xms512m)
>> > > > > > >>>>>> CPU: i7 2.2 Ghz.
>> > > > > > >>>>>>
>> > > > > > >>>>>> Note: I am using a slightly-modified, directly-accessible
>> > > Kafka
>> > > > > > >> Streams
>> > > > > > >>>>>> RocksDB
>> > > > > > >>>>>> implementation (RocksDB.java, basically just avoiding the
>> > > > > > >>>>>> ProcessorContext).
>> > > > > > >>>>>> There are no modifications to the default RocksDB values
>> > > > provided
>> > > > > in
>> > > > > > >> the
>> > > > > > >>>>>> 2.1/trunk release.
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>> keysize = 128 bytes
>> > > > > > >>>>>> valsize = 512 bytes
>> > > > > > >>>>>>
>> > > > > > >>>>>> Step 1:
>> > > > > > >>>>>> Write X positive matching events: (key = prefix +
>> > left-padded
>> > > > > > >>>>>> auto-incrementing integer)
>> > > > > > >>>>>> Step 2:
>> > > > > > >>>>>> Write 10X negative matching events (key = left-padded
>> > > > > > >> auto-incrementing
>> > > > > > >>>>>> integer)
>> > > > > > >>>>>> Step 3:
>> > > > > > >>>>>> Perform flush
>> > > > > > >>>>>> Step 4:
>> > > > > > >>>>>> Perform prefixScan
>> > > > > > >>>>>> Step 5:
>> > > > > > >>>>>> Iterate through return Iterator and validate the count of
>> > > > expected
>> > > > > > >>>>> events.
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> ============================================================================================
>> > > > > > >>>>>> Results:
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> ============================================================================================
>> > > > > > >>>>>> X = 1k (11k events total)
>> > > > > > >>>>>> Flush Time = 39 mS
>> > > > > > >>>>>> Scan Time = 7 mS
>> > > > > > >>>>>> 6.9 MB disk
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> --------------------------------------------------------------------------------------------
>> > > > > > >>>>>> X = 10k (110k events total)
>> > > > > > >>>>>> Flush Time = 45 mS
>> > > > > > >>>>>> Scan Time = 8 mS
>> > > > > > >>>>>> 127 MB
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> --------------------------------------------------------------------------------------------
>> > > > > > >>>>>> X = 100k (1.1M events total)
>> > > > > > >>>>>> Test1:
>> > > > > > >>>>>> Flush Time = 60 mS
>> > > > > > >>>>>> Scan Time = 12 mS
>> > > > > > >>>>>> 678 MB
>> > > > > > >>>>>>
>> > > > > > >>>>>> Test2:
>> > > > > > >>>>>> Flush Time = 45 mS
>> > > > > > >>>>>> Scan Time = 7 mS
>> > > > > > >>>>>> 576 MB
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> --------------------------------------------------------------------------------------------
>> > > > > > >>>>>> X = 1MB (11M events total)
>> > > > > > >>>>>> Test1:
>> > > > > > >>>>>> Flush Time = 52 mS
>> > > > > > >>>>>> Scan Time = 19 mS
>> > > > > > >>>>>> 7.2 GB
>> > > > > > >>>>>>
>> > > > > > >>>>>> Test2:
>> > > > > > >>>>>> Flush Time = 84 mS
>> > > > > > >>>>>> Scan Time = 34 mS
>> > > > > > >>>>>> 9.1 GB
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> --------------------------------------------------------------------------------------------
>> > > > > > >>>>>> X = 2.5M (27.5M events total)
>> > > > > > >>>>>> Test1:
>> > > > > > >>>>>> Flush Time = 82 mS
>> > > > > > >>>>>> Scan Time = 63 mS
>> > > > > > >>>>>> 17GB - 276 sst files
>> > > > > > >>>>>>
>> > > > > > >>>>>> Test2:
>> > > > > > >>>>>> Flush Time = 116 mS
>> > > > > > >>>>>> Scan Time = 35 mS
>> > > > > > >>>>>> 23GB - 361 sst files
>> > > > > > >>>>>>
>> > > > > > >>>>>> Test3:
>> > > > > > >>>>>> Flush Time = 103 mS
>> > > > > > >>>>>> Scan Time = 82 mS
>> > > > > > >>>>>> 19 GB - 300 sst files
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> --------------------------------------------------------------------------------------------
>> > > > > > >>>>>>
>> > > > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M
>> events. I
>> > > > tried
>> > > > > > to
>> > > > > > >> go
>> > > > > > >>>>>> to X = 10M (110M events) but RocksDB was going into the
>> > 100GB+
>> > > > > range
>> > > > > > >>>>> and my
>> > > > > > >>>>>> laptop ran out of disk. More extensive testing could be
>> done
>> > > > but I
>> > > > > > >>>>> suspect
>> > > > > > >>>>>> that it would be in line with what we're seeing in the
>> > results
>> > > > > > above.
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>> At this point in time, I think the only major discussion
>> > point
>> > > > is
>> > > > > > >> really
>> > > > > > >>>>>> around what Jan and I have disagreed on: repartitioning
>> > back +
>> > > > > > >> resolving
>> > > > > > >>>>>> potential out of order issues or leaving that up to the
>> > client
>> > > > to
>> > > > > > >>>>> handle.
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>> Thanks folks,
>> > > > > > >>>>>>
>> > > > > > >>>>>> Adam
>> > > > > > >>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
>> > > > > > Jan.Filipiak@trivago.com
>> > > > > > >>>
>> > > > > > >>>>>> wrote:
>> > > > > > >>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>> > > > > > >>>>>>>> Hi all,
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Sorry that this discussion petered out... I think the
>> 2.1
>> > > > > release
>> > > > > > >>>>>>> caused an
>> > > > > > >>>>>>>> extended distraction that pushed it off everyone's
>> radar
>> > > > (which
>> > > > > > was
>> > > > > > >>>>>>>> precisely Adam's concern). Personally, I've also had
>> some
>> > > > extend
>> > > > > > >>>>>>>> distractions of my own that kept (and continue to
>> keep) me
>> > > > > > >>>>> preoccupied.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> However, calling for a vote did wake me up, so I guess
>> Jan
>> > > was
>> > > > > on
>> > > > > > >> the
>> > > > > > >>>>>>> right
>> > > > > > >>>>>>>> track!
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> I've gone back and reviewed the whole KIP document and
>> the
>> > > > prior
>> > > > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> API Thoughts:
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 1. If I read the KIP right, you are proposing a
>> > many-to-one
>> > > > > join.
>> > > > > > >>>>> Could
>> > > > > > >>>>>>> we
>> > > > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
>> flip
>> > > the
>> > > > > > design
>> > > > > > >>>>>>> around
>> > > > > > >>>>>>>> and make it a oneToManyJoin?
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join
>> > > type,
>> > > > > and
>> > > > > > it
>> > > > > > >>>>>>> seems
>> > > > > > >>>>>>>> like it might trick some people into using it for a
>> > > one-to-one
>> > > > > > join.
>> > > > > > >>>>>>> This
>> > > > > > >>>>>>>> would work, of course, but it would be super
>> inefficient
>> > > > > compared
>> > > > > > to
>> > > > > > >>>>> a
>> > > > > > >>>>>>>> simple rekey-and-join.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 2. I might have missed it, but I don't think it's
>> > specified
>> > > > > > whether
>> > > > > > >>>>>>> it's an
>> > > > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer
>> join, as
>> > > > > > >>>>> (neglecting
>> > > > > > >>>>>>> IQ),
>> > > > > > >>>>>>>> the rest can be achieved by filtering or by handling
>> it in
>> > > the
>> > > > > > >>>>>>> ValueJoiner.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite
>> > > right.
>> > > > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
>> > > paradigms
>> > > > in
>> > > > > > >>>>> play in
>> > > > > > >>>>>>>> the Streams API, so it's confusing, but instead of
>> three
>> > > > > > Serialized
>> > > > > > >>>>>>> args, I
>> > > > > > >>>>>>>> think it would be better to have one that allows
>> > > (optionally)
>> > > > > > >> setting
>> > > > > > >>>>>>> the 4
>> > > > > > >>>>>>>> incoming serdes. The result serde is defined by the
>> > > > > Materialized.
>> > > > > > >> The
>> > > > > > >>>>>>>> incoming serdes can be optional because they might
>> already
>> > > be
>> > > > > > >>>>> available
>> > > > > > >>>>>>> on
>> > > > > > >>>>>>>> the source KTables, or the default serdes from the
>> config
>> > > > might
>> > > > > be
>> > > > > > >>>>>>>> applicable.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins
>> > > don't
>> > > > > > allow
>> > > > > > >>>>>>> setting
>> > > > > > >>>>>>>> one, and it seems like it might actually be harmful,
>> since
>> > > the
>> > > > > > rekey
>> > > > > > >>>>>>>> operation needs to produce results that are
>> co-partitioned
>> > > > with
>> > > > > > the
>> > > > > > >>>>>>> "other"
>> > > > > > >>>>>>>> KTable.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I
>> didn't
>> > > > > actually
>> > > > > > >>>>>>> follow
>> > > > > > >>>>>>>> what Matthias meant about namespacing requiring
>> > > > "deserializing"
>> > > > > > the
>> > > > > > >>>>>>> record
>> > > > > > >>>>>>>> header. The headers are already Strings, so I don't
>> think
>> > > that
>> > > > > > >>>>>>>> deserialization is required. If we applied the
>> namespace
>> > at
>> > > > > source
>> > > > > > >>>>> nodes
>> > > > > > >>>>>>>> and stripped it at sink nodes, this would be
>> practically
>> > no
>> > > > > > >> overhead.
>> > > > > > >>>>>>> The
>> > > > > > >>>>>>>> advantage of the namespace idea is that no public API
>> > change
>> > > > wrt
>> > > > > > >>>>> headers
>> > > > > > >>>>>>>> needs to happen, and no restrictions need to be placed
>> on
>> > > > users'
>> > > > > > >>>>>>> headers.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> (Although I'm wondering if we can get away without the
>> > > header
>> > > > at
>> > > > > > >>>>> all...
>> > > > > > >>>>>>>> stay tuned)
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM
>> table
>> > > > > growing
>> > > > > > >>>>>>> without
>> > > > > > >>>>>>>> bound. As I read it, the HWM table is effectively
>> > > implementing
>> > > > > OCC
>> > > > > > >> to
>> > > > > > >>>>>>>> resolve the problem you noted with disordering when the
>> > > rekey
>> > > > is
>> > > > > > >>>>>>>> reversed... particularly notable when the FK changes.
>> As
>> > > such,
>> > > > > it
>> > > > > > >>>>> only
>> > > > > > >>>>>>>> needs to track the most recent "version" (the offset in
>> > the
>> > > > > source
>> > > > > > >>>>>>>> partition) of each key. Therefore, it should have the
>> same
>> > > > > number
>> > > > > > of
>> > > > > > >>>>>>> keys
>> > > > > > >>>>>>>> as the source table at all times.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> I see that you are aware of KIP-258, which I think
>> might
>> > be
>> > > > > > relevant
>> > > > > > >>>>> in
>> > > > > > >>>>>>> a
>> > > > > > >>>>>>>> couple of ways. One: it's just about storing the
>> timestamp
>> > > in
>> > > > > the
>> > > > > > >>>>> state
>> > > > > > >>>>>>>> store, but the ultimate idea is to effectively use the
>> > > > timestamp
>> > > > > > as
>> > > > > > >>>>> an
>> > > > > > >>>>>>> OCC
>> > > > > > >>>>>>>> "version" to drop disordered updates. You wouldn't
>> want to
>> > > use
>> > > > > the
>> > > > > > >>>>>>>> timestamp for this operation, but if you were to use a
>> > > similar
>> > > > > > >>>>>>> mechanism to
>> > > > > > >>>>>>>> store the source offset in the store alongside the
>> > re-keyed
>> > > > > > values,
>> > > > > > >>>>> then
>> > > > > > >>>>>>>> you could avoid a separate table.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 6. You and Jan have been thinking about this for a long
>> > > time,
>> > > > so
>> > > > > > >> I've
>> > > > > > >>>>>>>> probably missed something here, but I'm wondering if we
>> > can
>> > > > > avoid
>> > > > > > >> the
>> > > > > > >>>>>>> HWM
>> > > > > > >>>>>>>> tracking at all and resolve out-of-order during a final
>> > join
>> > > > > > >>>>> instead...
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Let's say we're joining a left table (Integer K: Letter
>> > FK,
>> > > > > (other
>> > > > > > >>>>>>> data))
>> > > > > > >>>>>>>> to a right table (Letter K: (some data)).
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Left table:
>> > > > > > >>>>>>>> 1: (A, xyz)
>> > > > > > >>>>>>>> 2: (B, asd)
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Right table:
>> > > > > > >>>>>>>> A: EntityA
>> > > > > > >>>>>>>> B: EntityB
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> We could do a rekey as you proposed with a combined
>> key,
>> > but
>> > > > not
>> > > > > > >>>>>>>> propagating the value at all..
>> > > > > > >>>>>>>> Rekey table:
>> > > > > > >>>>>>>> A-1: (dummy value)
>> > > > > > >>>>>>>> B-2: (dummy value)
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Which we then join with the right table to produce:
>> > > > > > >>>>>>>> A-1: EntityA
>> > > > > > >>>>>>>> B-2: EntityB
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Which gets rekeyed back:
>> > > > > > >>>>>>>> 1: A, EntityA
>> > > > > > >>>>>>>> 2: B, EntityB
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> And finally we do the actual join:
>> > > > > > >>>>>>>> Result table:
>> > > > > > >>>>>>>> 1: ((A, xyz), EntityA)
>> > > > > > >>>>>>>> 2: ((B, asd), EntityB)
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> The thing is that in that last join, we have the
>> > opportunity
>> > > > to
>> > > > > > >>>>> compare
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>> current FK in the left table with the incoming PK of
>> the
>> > > right
>> > > > > > >>>>> table. If
>> > > > > > >>>>>>>> they don't match, we just drop the event, since it
>> must be
>> > > > > > outdated.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz)
>> gets
>> > > > > updated
>> > > > > > >> to
>> > > > > > >>>>>>> (1:
>> > > > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether
>> the
>> > > > final
>> > > > > > >>>>> state
>> > > > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the
>> > algorithm
>> > > > > above,
>> > > > > > >>>>> you
>> > > > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1:
>> (B,
>> > > xyz),
>> > > > > (B,
>> > > > > > >>>>>>>> EntityB)). It seems like this does give you enough
>> > > information
>> > > > > to
>> > > > > > >>>>> make
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>> right choice, regardless of disordering.
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> Will check Adams patch, but this should work. As
>> mentioned
>> > > > often
>> > > > > I
>> > > > > > am
>> > > > > > >>>>>>> not convinced on partitioning back for the user
>> > > automatically.
>> > > > I
>> > > > > > >> think
>> > > > > > >>>>>>> this is the real performance eater ;)
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
>> > > > performance
>> > > > > of
>> > > > > > >>>>> the
>> > > > > > >>>>>>>> range scans when records change in the right table.
>> You've
>> > > > said
>> > > > > > that
>> > > > > > >>>>>>> you've
>> > > > > > >>>>>>>> been using the algorithm you presented in production
>> for a
>> > > > > while.
>> > > > > > >> Can
>> > > > > > >>>>>>> you
>> > > > > > >>>>>>>> give us a sense of the performance characteristics
>> you've
>> > > > > > observed?
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> Make it work, make it fast, make it beautiful. The
>> topmost
>> > > > thing
>> > > > > > here
>> > > > > > >>>>> is
>> > > > > > >>>>>>> / was correctness. In practice I do not measure the
>> > > performance
>> > > > > of
>> > > > > > >> the
>> > > > > > >>>>>>> range scan. Usual cases I run this with is emitting
>> 500k -
>> > > 1kk
>> > > > > rows
>> > > > > > >>>>>>> on a left hand side change. The range scan is just the
>> work
>> > > you
>> > > > > > gotta
>> > > > > > >>>>>>> do, also when you pack your data into different formats,
>> > > > usually
>> > > > > > the
>> > > > > > >>>>>>> rocks performance is very tight to the size of the data
>> and
>> > > we
>> > > > > > can't
>> > > > > > >>>>>>> really change that. It is more important for users to
>> > prevent
>> > > > > > useless
>> > > > > > >>>>>>> updates to begin with. My left hand side is guarded to
>> drop
>> > > > > changes
>> > > > > > >>>>> that
>> > > > > > >>>>>>> are not going to change my join output.
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> usually it's:
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> drop unused fields and then don't forward if
>> > old.equals(new)
>> > > > > > >>>>>>>
>> > > > > > >>>>>>> regarding to the performance of creating an iterator for
>> > > > smaller
>> > > > > > >>>>>>> fanouts, users can still just do a group by first then
>> > > anyways.
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>>> I could only think of one alternative, but I'm not
>> sure if
>> > > > it's
>> > > > > > >>>>> better
>> > > > > > >>>>>>> or
>> > > > > > >>>>>>>> worse... If the first re-key only needs to preserve the
>> > > > original
>> > > > > > >> key,
>> > > > > > >>>>>>> as I
>> > > > > > >>>>>>>> proposed in #6, then we could store a vector of keys in
>> > the
>> > > > > value:
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Left table:
>> > > > > > >>>>>>>> 1: A,...
>> > > > > > >>>>>>>> 2: B,...
>> > > > > > >>>>>>>> 3: A,...
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Gets re-keyed:
>> > > > > > >>>>>>>> A: [1, 3]
>> > > > > > >>>>>>>> B: [2]
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Then, the rhs part of the join would only need a
>> regular
>> > > > > > single-key
>> > > > > > >>>>>>> lookup.
>> > > > > > >>>>>>>> Of course we have to deal with the problem of large
>> > values,
>> > > as
>> > > > > > >>>>> there's
>> > > > > > >>>>>>> no
>> > > > > > >>>>>>>> bound on the number of lhs records that can reference
>> rhs
>> > > > > records.
>> > > > > > >>>>>>> Offhand,
>> > > > > > >>>>>>>> I'd say we could page the values, so when one row is
>> past
>> > > the
>> > > > > > >>>>>>> threshold, we
>> > > > > > >>>>>>>> append the key for the next page. Then in most cases,
>> it
>> > > would
>> > > > > be
>> > > > > > a
>> > > > > > >>>>>>> single
>> > > > > > >>>>>>>> key lookup, but for large fan-out updates, it would be
>> one
>> > > per
>> > > > > > (max
>> > > > > > >>>>>>> value
>> > > > > > >>>>>>>> size)/(avg lhs key size).
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> This seems more complex, though... Plus, I think
>> there's
>> > > some
>> > > > > > extra
>> > > > > > >>>>>>>> tracking we'd need to do to know when to emit a
>> > retraction.
>> > > > For
>> > > > > > >>>>> example,
>> > > > > > >>>>>>>> when record 1 is deleted, the re-key table would just
>> have
>> > > (A:
>> > > > > > [3]).
>> > > > > > >>>>>>> Some
>> > > > > > >>>>>>>> kind of tombstone is needed so that the join result
>> for 1
>> > > can
>> > > > > also
>> > > > > > >> be
>> > > > > > >>>>>>>> retracted.
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> That's all!
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful
>> > KIP.
>> > > > > Sorry
>> > > > > > >> the
>> > > > > > >>>>>>>> discussion has been slow.
>> > > > > > >>>>>>>> -John
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>> > > > > > >>>>> Jan.Filipiak@trivago.com>
>> > > > > > >>>>>>>> wrote:
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>>> Id say you can just call the vote.
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>>> that happens all the time, and if something comes up,
>> it
>> > > just
>> > > > > > goes
>> > > > > > >>>>> back
>> > > > > > >>>>>>>>> to discuss.
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>>> would not expect to much attention with another
>> another
>> > > email
>> > > > > in
>> > > > > > >>>>> this
>> > > > > > >>>>>>>>> thread.
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>>> best Jan
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>> > > > > > >>>>>>>>>> Hello Contributors
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do
>> need
>> > to
>> > > > bump
>> > > > > > >>>>> this to
>> > > > > > >>>>>>>>> keep
>> > > > > > >>>>>>>>>> visibility up. I am still intending to push this
>> through
>> > > > once
>> > > > > > >>>>>>> contributor
>> > > > > > >>>>>>>>>> feedback is given.
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> Main points that need addressing:
>> > > > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
>> > > singular
>> > > > > > graph
>> > > > > > >>>>> node
>> > > > > > >>>>>>>>> into
>> > > > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right
>> > > now. I
>> > > > > am
>> > > > > > a
>> > > > > > >>>>> bit
>> > > > > > >>>>>>>>> fuzzy
>> > > > > > >>>>>>>>>> on how the optimizations are supposed to work, so I
>> > would
>> > > > > > >>>>> appreciate
>> > > > > > >>>>>>> any
>> > > > > > >>>>>>>>>> help on this aspect.
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This
>> thread
>> > > has
>> > > > > > much
>> > > > > > >>>>>>>>> discourse
>> > > > > > >>>>>>>>>> between Jan and I between the current highwater mark
>> > > > proposal
>> > > > > > and
>> > > > > > >> a
>> > > > > > >>>>>>>>> groupBy
>> > > > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need
>> to
>> > > > > strictly
>> > > > > > >>>>> handle
>> > > > > > >>>>>>>>> any
>> > > > > > >>>>>>>>>> chance of out-of-order data and leave none of it up
>> to
>> > the
>> > > > > > >>>>> consumer.
>> > > > > > >>>>>>> Any
>> > > > > > >>>>>>>>>> comments or suggestions here would also help.
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> 3) Anything else that you see that would prevent this
>> > from
>> > > > > > moving
>> > > > > > >>>>> to a
>> > > > > > >>>>>>>>> vote?
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> Thanks
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> Adam
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> > > > > > >>>>>>>>> adam.bellemare@gmail.com>
>> > > > > > >>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>>> Hi Jan
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
>> > > > > > >>>>> Stores.persistentWindowStore,
>> > > > > > >>>>>>> you
>> > > > > > >>>>>>>>>>> actually only need to specify the amount of segments
>> > you
>> > > > want
>> > > > > > and
>> > > > > > >>>>> how
>> > > > > > >>>>>>>>> large
>> > > > > > >>>>>>>>>>> they are. To the best of my understanding, what
>> happens
>> > > is
>> > > > > that
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>>>> segments are automatically rolled over as new data
>> with
>> > > new
>> > > > > > >>>>>>> timestamps
>> > > > > > >>>>>>>>> are
>> > > > > > >>>>>>>>>>> created. We use this exact functionality in some of
>> the
>> > > > work
>> > > > > > done
>> > > > > > >>>>>>>>>>> internally at my company. For reference, this is the
>> > > > hopping
>> > > > > > >>>>> windowed
>> > > > > > >>>>>>>>> store.
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>> In the code that I have provided, there are going
>> to be
>> > > two
>> > > > > 24h
>> > > > > > >>>>>>>>> segments.
>> > > > > > >>>>>>>>>>> When a record is put into the windowStore, it will
>> be
>> > > > > inserted
>> > > > > > at
>> > > > > > >>>>>>> time
>> > > > > > >>>>>>>>> T in
>> > > > > > >>>>>>>>>>> both segments. The two segments will always overlap
>> by
>> > > 12h.
>> > > > > As
>> > > > > > >>>>> time
>> > > > > > >>>>>>>>> goes on
>> > > > > > >>>>>>>>>>> and new records are added (say at time T+12h+), the
>> > > oldest
>> > > > > > >> segment
>> > > > > > >>>>>>> will
>> > > > > > >>>>>>>>> be
>> > > > > > >>>>>>>>>>> automatically deleted and a new segment created. The
>> > > > records
>> > > > > > are
>> > > > > > >>>>> by
>> > > > > > >>>>>>>>> default
>> > > > > > >>>>>>>>>>> inserted with the context.timestamp(), such that it
>> is
>> > > the
>> > > > > > record
>> > > > > > >>>>>>> time,
>> > > > > > >>>>>>>>> not
>> > > > > > >>>>>>>>>>> the clock time, which is used.
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>> To the best of my understanding, the timestamps are
>> > > > retained
>> > > > > > when
>> > > > > > >>>>>>>>>>> restoring from the changelog.
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL
>> > at a
>> > > > > > >>>>>>> segment-level,
>> > > > > > >>>>>>>>>>> instead of at an individual record level.
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>> > > > > > >>>>>>> Jan.Filipiak@trivago.com>
>> > > > > > >>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
>> > > > > > ClassCastException
>> > > > > > >>>>> or
>> > > > > > >>>>>>>>>>>> similar.
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> You either would have to specify the window you
>> > > fetch/put
>> > > > or
>> > > > > > >>>>> iterate
>> > > > > > >>>>>>>>>>>> across all windows the key was found in right?
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> I just hope the window-store doesn't check
>> stream-time
>> > > > under
>> > > > > > the
>> > > > > > >>>>>>> hoods
>> > > > > > >>>>>>>>>>>> that would be a questionable interface.
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> If it does: did you see my comment on checking all
>> the
>> > > > > windows
>> > > > > > >>>>>>> earlier?
>> > > > > > >>>>>>>>>>>> that would be needed to actually give reasonable
>> time
>> > > > > > gurantees.
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> Best
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> > > > > > >>>>>>>>>>>>> Hi Jan
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
>> changed
>> > > the
>> > > > > > state
>> > > > > > >>>>>>> store,
>> > > > > > >>>>>>>>>>>> not
>> > > > > > >>>>>>>>>>>>> the ProcessorSupplier.
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>> Thanks,
>> > > > > > >>>>>>>>>>>>> Adam
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> @Guozhang
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed
>> > something
>> > > > that
>> > > > > > >>>>> will be
>> > > > > > >>>>>>>>>>>>>>> extremely
>> > > > > > >>>>>>>>>>>>>>> useful for this KIP.
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> @Jan
>> > > > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I
>> > will
>> > > > not
>> > > > > > be
>> > > > > > >>>>>>> moving
>> > > > > > >>>>>>>>>>>> ahead
>> > > > > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy
>> > > solution
>> > > > > as
>> > > > > > >> you
>> > > > > > >>>>>>>>>>>> propose.
>> > > > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it
>> > yourself
>> > > > off
>> > > > > > of
>> > > > > > >>>>> my
>> > > > > > >>>>>>>>>>>> current PR
>> > > > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I
>> would
>> > > be
>> > > > > more
>> > > > > > >>>>> than
>> > > > > > >>>>>>>>>>>> happy to
>> > > > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it
>> > stands
>> > > > > right
>> > > > > > >>>>> now,
>> > > > > > >>>>>>> I do
>> > > > > > >>>>>>>>>>>> not
>> > > > > > >>>>>>>>>>>>>>> really have more time to invest into
>> alternatives
>> > > > without
>> > > > > > >>>>> there
>> > > > > > >>>>>>>>> being
>> > > > > > >>>>>>>>>>>> a
>> > > > > > >>>>>>>>>>>>>>> strong indication from the binding voters which
>> > they
>> > > > > would
>> > > > > > >>>>>>> prefer.
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave
>> up
>> > on
>> > > > the
>> > > > > > >>>>> streams
>> > > > > > >>>>>>>>> DSL
>> > > > > > >>>>>>>>>>>> for
>> > > > > > >>>>>>>>>>>>>> some time already, otherwise I would have pulled
>> > this
>> > > > KIP
>> > > > > > >>>>> through
>> > > > > > >>>>>>>>>>>> already.
>> > > > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on
>> > > PAPI.
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the
>> windowed
>> > > > state
>> > > > > > >>>>> store
>> > > > > > >>>>>>> in
>> > > > > > >>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>> next
>> > > > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I
>> > will
>> > > > come
>> > > > > > >> back
>> > > > > > >>>>>>> for
>> > > > > > >>>>>>>>>>>> final
>> > > > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of
>> > the
>> > > > > > binding
>> > > > > > >>>>>>> voters
>> > > > > > >>>>>>>>>>>> could
>> > > > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
>> updated
>> > it
>> > > > > > >>>>> according
>> > > > > > >>>>>>> to
>> > > > > > >>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>> latest plan:
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> > > > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed
>> > > store.
>> > > > > > This
>> > > > > > >>>>>>> could
>> > > > > > >>>>>>>>> be
>> > > > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they
>> > are
>> > > > > > >>>>> completed.
>> > > > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> Thanks,
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> Adam
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
>> > already
>> > > > > > updated
>> > > > > > >>>>> in
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> PR?
>> > > > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
>> > > > > something?
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
>> > > > > > >>>>>>> wangguoz@gmail.com>
>> > > > > > >>>>>>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is
>> the
>> > > > wrong
>> > > > > > >> link,
>> > > > > > >>>>>>> as it
>> > > > > > >>>>>>>>>>>> is
>> > > > > > >>>>>>>>>>>>>>>> for
>> > > > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as
>> part of
>> > > > > KIP-258
>> > > > > > >>>>> we do
>> > > > > > >>>>>>>>>>>> want to
>> > > > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source
>> > KTable"
>> > > > such
>> > > > > > >> that
>> > > > > > >>>>>>>>>>>> instead of
>> > > > > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized
>> > store,
>> > > > > i.e.
>> > > > > > >>>>>>> following
>> > > > > > >>>>>>>>>>>>>>>> offset
>> > > > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older
>> > than
>> > > > the
>> > > > > > >>>>> current
>> > > > > > >>>>>>>>>>>> key's
>> > > > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>> Guozhang
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang
>> Wang <
>> > > > > > >>>>>>>>> wangguoz@gmail.com>
>> > > > > > >>>>>>>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>> Hello Adam,
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the
>> final
>> > > step
>> > > > > > (i.e.
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>> high
>> > > > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
>> with
>> > a
>> > > > > window
>> > > > > > >>>>>>> store),
>> > > > > > >>>>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>> think
>> > > > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually
>> help:
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > >>>>>>>>>>>>>>>>>
>> > 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a
>> key-value
>> > > > store
>> > > > > > >>>>> (i.e.
>> > > > > > >>>>>>> only
>> > > > > > >>>>>>>>>>>> for
>> > > > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
>> usage,
>> > as
>> > > > > > >>>>> described
>> > > > > > >>>>>>> in
>> > > > > > >>>>>>>>>>>>>>>>>
>> https://issues.apache.org/jira/browse/KAFKA-5533
>> > ,
>> > > is
>> > > > > > that
>> > > > > > >>>>> we
>> > > > > > >>>>>>> can
>> > > > > > >>>>>>>>>>>> then
>> > > > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
>> > > > > timestamp
>> > > > > > is
>> > > > > > >>>>>>>>> smaller
>> > > > > > >>>>>>>>>>>> than
>> > > > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I
>> > think
>> > > it
>> > > > > is
>> > > > > > >>>>> very
>> > > > > > >>>>>>>>>>>> similar to
>> > > > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
>> > > > > filtering,
>> > > > > > >>>>> while
>> > > > > > >>>>>>>>> you
>> > > > > > >>>>>>>>>>>> only
>> > > > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the
>> > > joining
>> > > > > > >> records
>> > > > > > >>>>>>> are
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>> correctly
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the
>> final
>> > > > stage.
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and
>> > hence
>> > > > > > >>>>>>> non-windowed
>> > > > > > >>>>>>>>>>>> KTables
>> > > > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not
>> really
>> > > have
>> > > > a
>> > > > > > good
>> > > > > > >>>>>>>>> support
>> > > > > > >>>>>>>>>>>> for
>> > > > > > >>>>>>>>>>>>>>>>> their joins anyways (
>> > > > > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> > > > > > >>>>>>>>>>>>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
>> > > KTable-KTable
>> > > > > > >>>>> non-key
>> > > > > > >>>>>>>>> joins
>> > > > > > >>>>>>>>>>>> for
>> > > > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> Guozhang
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak
>> <
>> > > > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would
>> > grow
>> > > > > > >> endlessly
>> > > > > > >>>>>>> based
>> > > > > > >>>>>>>>>>>> on
>> > > > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair
>> of
>> > > > (<this
>> > > > > > >>>>> table
>> > > > > > >>>>>>>>>>>> primary
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> key>,
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is
>> used
>> > > to
>> > > > > > >>>>>>> differentiate
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> between
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
>> proposal
>> > > > would
>> > > > > > be
>> > > > > > >>>>> to
>> > > > > > >>>>>>>>>>>> replace
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> it
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N.
>> This
>> > > would
>> > > > > > allow
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>> same
>> > > > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time.
>> This
>> > > > > should
>> > > > > > >>>>> allow
>> > > > > > >>>>>>> for
>> > > > > > >>>>>>>>>>>> all
>> > > > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
>> > should
>> > > be
>> > > > > > >>>>>>> customizable
>> > > > > > >>>>>>>>>>>> by
>> > > > > > >>>>>>>>>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
>> perhaps
>> > > just
>> > > > > 10
>> > > > > > >>>>>>> minutes
>> > > > > > >>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> window,
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do
>> the
>> > > > trick
>> > > > > > >> here.
>> > > > > > >>>>>>> Even
>> > > > > > >>>>>>>>>>>> if I
>> > > > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
>> > > repartitioning
>> > > > > > >>>>> optional
>> > > > > > >>>>>>>>>>>> since I
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I
>> am a
>> > > > little
>> > > > > > bit
>> > > > > > >>>>>>>>>>>> sceptical
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> about
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one
>> > > could
>> > > > > run
>> > > > > > >>>>> into
>> > > > > > >>>>>>>>>>>> problems
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> when
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window
>> border. I
>> > > will
>> > > > > > check
>> > > > > > >>>>> you
>> > > > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its
>> problematic, we
>> > > > could
>> > > > > > >>>>> still
>> > > > > > >>>>>>>>> check
>> > > > > > >>>>>>>>>>>>>>>>>> _all_
>> > > > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance
>> > > impact I
>> > > > > > >> guess.
>> > > > > > >>>>>>> Will
>> > > > > > >>>>>>>>>>>> let
>> > > > > > >>>>>>>>>>>>>>>>>> you
>> > > > > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct
>> as
>> > > is. I
>> > > > > > >>>>> wouldn't
>> > > > > > >>>>>>> not
>> > > > > > >>>>>>>>>>>> like
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
>> > > timestamp(A)  <
>> > > > > > >>>>>>>>> timestamp(B).
>> > > > > > >>>>>>>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> think
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> we can't expect that.
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> @Jan
>> > > > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now -
>> > thanks
>> > > > for
>> > > > > > the
>> > > > > > >>>>>>>>>>>> diagram, it
>> > > > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do
>> not
>> > > have
>> > > > > the
>> > > > > > >>>>>>> original
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> primary
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
>> > > available
>> > > > > > then
>> > > > > > >>>>> you
>> > > > > > >>>>>>>>>>>> would be
>> > > > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map.
>> > That
>> > > > > being
>> > > > > > >>>>> said,
>> > > > > > >>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> encourage
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for
>> > > clarity
>> > > > > for
>> > > > > > >>>>>>> everyone
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> else.
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really
>> hard
>> > > > work.
>> > > > > > But
>> > > > > > >>>>> I
>> > > > > > >>>>>>>>>>>> understand
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
>> > > original
>> > > > > > >> primary
>> > > > > > >>>>>>> key,
>> > > > > > >>>>>>>>> We
>> > > > > > >>>>>>>>>>>>>>>>>> have
>> > > > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI
>> > and
>> > > > > > >> basically
>> > > > > > >>>>>>> not
>> > > > > > >>>>>>>>>>>> using
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> any
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed
>> > that
>> > > in
>> > > > > > >>>>> original
>> > > > > > >>>>>>> DSL
>> > > > > > >>>>>>>>>>>> its
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> not
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess
>> up on
>> > > my
>> > > > > end.
>> > > > > > >>>>> Will
>> > > > > > >>>>>>>>>>>> finish
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
>> week.
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the
>> Map
>> > > stay
>> > > > > > >> inside
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> State
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes
>> > have
>> > > > > > >>>>> propagated?
>> > > > > > >>>>>>>>> Isn't
>> > > > > > >>>>>>>>>>>>>>>>>>> this
>> > > > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
>> state
>> > > > store?
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
>> substractor
>> > is
>> > > > > gonna
>> > > > > > >>>>>>> return
>> > > > > > >>>>>>>>>>>> `null`
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But
>> there
>> > is
>> > > > > going
>> > > > > > to
>> > > > > > >>>>> be
>> > > > > > >>>>>>> a
>> > > > > > >>>>>>>>>>>> store
>> > > > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this
>> > store
>> > > > > > directly
>> > > > > > >>>>> for
>> > > > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a
>> > > > regular
>> > > > > > >>>>> store,
>> > > > > > >>>>>>>>>>>> satisfying
>> > > > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
>> join.
>> > > The
>> > > > > > >>>>> Windowed
>> > > > > > >>>>>>>>>>>> store is
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> not
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull
>> > > > operation
>> > > > > > we
>> > > > > > >>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we
>> have
>> > the
>> > > > > > window
>> > > > > > >>>>>>> store
>> > > > > > >>>>>>>>>>>> also
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> have
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> the values then.
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom
>> > group
>> > > > by
>> > > > > > >>>>> before
>> > > > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i
>> > think
>> > > > it
>> > > > > > >> would
>> > > > > > >>>>>>> help
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> users
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the
>> > > > original
>> > > > > > >>>>> primary
>> > > > > > >>>>>>>>> key
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> issue I
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid
>> foundation
>> > > to
>> > > > > > build
>> > > > > > >>>>> on.
>> > > > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user.
>> > very
>> > > > > > >>>>>>> unfortunate. I
>> > > > > > >>>>>>>>>>>> could
>> > > > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do
>> not
>> > > > think
>> > > > > > its
>> > > > > > >>>>> a
>> > > > > > >>>>>>> good
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> decision.
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> Thanks
>> > > > > > >>>>>>>>>>>>>>>>>>> Adam
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta
>> > > Dumbre <
>> > > > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>> > > > > > >>>>>>> dumbreprajakta311@gmail.com
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM
>> Jan
>> > > > > Filipiak
>> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> <mailto:
>> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
>> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make
>> such a
>> > > > > chart.
>> > > > > > >> last
>> > > > > > >>>>>>> time i
>> > > > > > >>>>>>>>>>>> didn't
>> > > > > > >>>>>>>>>>>>>>>>>>>           get along
>> > > > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all
>> your
>> > > > > charts.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done
>> today
>> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam
>> Bellemare
>> > > > > wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of
>> > what I
>> > > > > > >> attempted
>> > > > > > >>>>> on
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> KIP.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>
>> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>
>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> > > > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> > > > > > >>>>>>>>>>>>>>>>>>>           <
>> > > > > > >>>>>>>>>>>>
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>
>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> > > > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the
>> > start
>> > > of
>> > > > > my
>> > > > > > own
>> > > > > > >>>>>>>>>>>> implementation
>> > > > > > >>>>>>>>>>>>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>           this
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could
>> not
>> > get
>> > > > it
>> > > > > to
>> > > > > > >>>>> work I
>> > > > > > >>>>>>> have
>> > > > > > >>>>>>>>>>>> since
>> > > > > > >>>>>>>>>>>>>>>>>>>           discarded the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time,
>> if
>> > you
>> > > > > wish
>> > > > > > to
>> > > > > > >>>>>>> continue
>> > > > > > >>>>>>>>>>>> pursuing
>> > > > > > >>>>>>>>>>>>>>>>>>>           for your
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that
>> you
>> > > > please
>> > > > > > >>>>> create a
>> > > > > > >>>>>>>>>>>> diagram on
>> > > > > > >>>>>>>>>>>>>>>>>>>           the KIP
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your
>> > solution.
>> > > > > > Please
>> > > > > > >>>>> feel
>> > > > > > >>>>>>> free
>> > > > > > >>>>>>>>> to
>> > > > > > >>>>>>>>>>>> use
>> > > > > > >>>>>>>>>>>>>>>>>>>           the image I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting
>> point.
>> > I
>> > > am
>> > > > > > having
>> > > > > > >>>>>>> trouble
>> > > > > > >>>>>>>>>>>>>>>>>>>           understanding your
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that
>> a
>> > > > > carefully
>> > > > > > >>>>>>> constructed
>> > > > > > >>>>>>>>>>>> diagram
>> > > > > > >>>>>>>>>>>>>>>>>>>           will clear
>> > > > > > >>>>>>>>>>>>>>>>>>>           > up
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings.
>> > Alternately,
>> > > > > > please
>> > > > > > >>>>> post a
>> > > > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only
>> guess
>> > at
>> > > > what
>> > > > > > you
>> > > > > > >>>>>>> mean, and
>> > > > > > >>>>>>>>>>>> since I
>> > > > > > >>>>>>>>>>>>>>>>>>>           value my
>> > > > > > >>>>>>>>>>>>>>>>>>>           > own
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value
>> yours,
>> > I
>> > > > > > believe
>> > > > > > >> it
>> > > > > > >>>>> is
>> > > > > > >>>>>>> your
>> > > > > > >>>>>>>>>>>>>>>>>>>           responsibility to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation
>> instead
>> > > of
>> > > > me
>> > > > > > >>>>> trying to
>> > > > > > >>>>>>>>> guess.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > Adam
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00
>> AM,
>> > Jan
>> > > > > > Filipiak
>> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> <mailto:
>> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing
>> > interested.
>> > > > > kafka
>> > > > > > >>>>>>> streams at
>> > > > > > >>>>>>>>>>>> this
>> > > > > > >>>>>>>>>>>>>>>>>>>           point supports
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as
>> > both
>> > > > > > streams
>> > > > > > >>>>> have
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> same
>> > > > > > >>>>>>>>>>>>>>>>>>> key.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently
>> implementing a
>> > > > join
>> > > > > > >> where a
>> > > > > > >>>>>>> KTable
>> > > > > > >>>>>>>>>>>> and a
>> > > > > > >>>>>>>>>>>>>>>>>>>           KTable can
>> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship
>> > (1:n).
>> > > > We
>> > > > > > >> exploit
>> > > > > > >>>>>>> that
>> > > > > > >>>>>>>>>>>> rocksdb
>> > > > > > >>>>>>>>>>>>>>>>>>> is
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> a
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data
>> sorted
>> > (At
>> > > > > least
>> > > > > > >>>>>>> exposes an
>> > > > > > >>>>>>>>>>>> API to
>> > > > > > >>>>>>>>>>>>>>>>>>>           access the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted
>> > fashion).
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats
>> > are
>> > > > well
>> > > > > > >>>>>>> understood
>> > > > > > >>>>>>>>> now
>> > > > > > >>>>>>>>>>>> and we
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> are
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > basically
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API
>> > Design
>> > > (
>> > > > > when
>> > > > > > >> Adam
>> > > > > > >>>>>>> sees
>> > > > > > >>>>>>>>> my
>> > > > > > >>>>>>>>>>>> newest
>> > > > > > >>>>>>>>>>>>>>>>>>>           message).
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track
>> record of
>> > > > > loosing
>> > > > > > >>>>> those
>> > > > > > >>>>>>> kinda
>> > > > > > >>>>>>>>>>>>>>>>>>>           arguments within
>> > > > > > >>>>>>>>>>>>>>>>>>>           > the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have
>> no
>> > > clue
>> > > > > > why.
>> > > > > > >> So
>> > > > > > >>>>> I
>> > > > > > >>>>>>>>>>>> literally
>> > > > > > >>>>>>>>>>>>>>>>>>>           can't wait for
>> > > > > > >>>>>>>>>>>>>>>>>>>           > you
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread
>> and
>> > > > give
>> > > > > > you
>> > > > > > >>>>>>> opinion on
>> > > > > > >>>>>>>>>>>> how we
>> > > > > > >>>>>>>>>>>>>>>>>>>           should
>> > > > > > >>>>>>>>>>>>>>>>>>>           > design
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
>> > > oneToManyJoin
>> > > > > and
>> > > > > > >> how
>> > > > > > >>>>>>> many
>> > > > > > >>>>>>>>>>>> power we
>> > > > > > >>>>>>>>>>>>>>>>>>>           want to give
>> > > > > > >>>>>>>>>>>>>>>>>>>           > to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity"
>> (where
>> > > > > > simplicity
>> > > > > > >>>>> isn't
>> > > > > > >>>>>>>>>>>> really that
>> > > > > > >>>>>>>>>>>>>>>>>>>           as users
>> > > > > > >>>>>>>>>>>>>>>>>>>           > still
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I
>> argue)
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on
>> > the
>> > > > > > >> discussion
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James
>> Kwan
>> > > > wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I
>> > > found
>> > > > > this
>> > > > > > >>>>> subject
>> > > > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
>> > > > > > >>>>>>>>>>>>>>>>>>>           > like
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a
>> > join
>> > > > > > table of
>> > > > > > >>>>> two
>> > > > > > >>>>>>>>>>>> streams? Is
>> > > > > > >>>>>>>>>>>>>>>>>>> there
>> > > > > > >>>>>>>>>>>>>>>>>>>           > somewhere
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
>> > > requirement
>> > > > or
>> > > > > > >>>>> proposal?
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM,
>> Jan
>> > > > > > Filipiak
>> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com
>> <mailto:
>> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
>> > > > Bellemare
>> > > > > > >> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing
>> using a
>> > > > > > Windowed
>> > > > > > >>>>> Store
>> > > > > > >>>>>>> to
>> > > > > > >>>>>>>>>>>> store the
>> > > > > > >>>>>>>>>>>>>>>>>>>           highwater
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this
>> > should
>> > > > work
>> > > > > > >> fine,
>> > > > > > >>>>>>> with
>> > > > > > >>>>>>>>> the
>> > > > > > >>>>>>>>>>>> caveat
>> > > > > > >>>>>>>>>>>>>>>>>>>           being that
>> > > > > > >>>>>>>>>>>>>>>>>>>           > it
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order
>> > > arrival
>> > > > > > for up
>> > > > > > >>>>> to
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> size of
>> > > > > > >>>>>>>>>>>>>>>>>>>           the window
>> > > > > > >>>>>>>>>>>>>>>>>>>           > (ie:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would
>> > > remove
>> > > > > the
>> > > > > > >>>>>>> possibility
>> > > > > > >>>>>>>>>>>> of it
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> being
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > unbounded
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
>> > > > suggestion, I
>> > > > > > >>>>> believe
>> > > > > > >>>>>>> this
>> > > > > > >>>>>>>>> is
>> > > > > > >>>>>>>>>>>> where
>> > > > > > >>>>>>>>>>>>>>>>>>>           we will
>> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement.
>> > While I
>> > > > do
>> > > > > > not
>> > > > > > >>>>>>> disagree
>> > > > > > >>>>>>>>>>>> with your
>> > > > > > >>>>>>>>>>>>>>>>>>>           statement
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be
>> additional
>> > > > joins
>> > > > > > done
>> > > > > > >>>>> in a
>> > > > > > >>>>>>>>>>>> real-world
>> > > > > > >>>>>>>>>>>>>>>>>>>           workflow, I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > do
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can
>> conclusively
>> > > deal
>> > > > > > with
>> > > > > > >>>>>>>>> out-of-order
>> > > > > > >>>>>>>>>>>>>>>>>>> arrival
>> > > > > > >>>>>>>>>>>>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent
>> > joins. I
>> > > > > have
>> > > > > > >>>>>>> attempted
>> > > > > > >>>>>>>>> what
>> > > > > > >>>>>>>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>>>>           think you have
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a
>> > high-water,
>> > > > > using
>> > > > > > >>>>>>> groupBy and
>> > > > > > >>>>>>>>>>>> reduce)
>> > > > > > >>>>>>>>>>>>>>>>>>>           and found
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes
>> too
>> > > > > quickly,
>> > > > > > or
>> > > > > > >>>>> the
>> > > > > > >>>>>>> load
>> > > > > > >>>>>>>>> on
>> > > > > > >>>>>>>>>>>> a
>> > > > > > >>>>>>>>>>>>>>>>>>>           stream thread
>> > > > > > >>>>>>>>>>>>>>>>>>>           > is
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages
>> > will
>> > > > > arrive
>> > > > > > >>>>>>>>> out-of-order
>> > > > > > >>>>>>>>>>>> and be
>> > > > > > >>>>>>>>>>>>>>>>>>>           incorrectly
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
>> > > > > intermediate
>> > > > > > >>>>> event
>> > > > > > >>>>>>> is
>> > > > > > >>>>>>>>>>>>>>>>>>> represented
>> > > > > > >>>>>>>>>>>>>>>>>>>           as the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > final
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on
>> > your
>> > > > > > groupBy
>> > > > > > >>>>>>>>>>>> implementation.
>> > > > > > >>>>>>>>>>>>>>>>>>>           There must be
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it
>> > is,
>> > > I
>> > > > > > would
>> > > > > > >>>>> just
>> > > > > > >>>>>>> like
>> > > > > > >>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it
>> must be
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up.
>> I
>> > > would
>> > > > > > like
>> > > > > > >> to
>> > > > > > >>>>>>> clarify
>> > > > > > >>>>>>>>>>>> before
>> > > > > > >>>>>>>>>>>>>>>>>>>           we draw a
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the
>> > scattered
>> > > > > events
>> > > > > > >>>>> back to
>> > > > > > >>>>>>>>> their
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> original
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I
>> > know
>> > > > how
>> > > > > > to
>> > > > > > >>>>>>>>> conclusively
>> > > > > > >>>>>>>>>>>> deal
>> > > > > > >>>>>>>>>>>>>>>>>>>           with
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a
>> > given
>> > > > time
>> > > > > > >> frame,
>> > > > > > >>>>>>> and to
>> > > > > > >>>>>>>>>>>> ensure
>> > > > > > >>>>>>>>>>>>>>>>>>>           that the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > data
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with
>> > the
>> > > > > input
>> > > > > > >>>>> events.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to
>> > share
>> > > > that
>> > > > > > >>>>>>> illustrates
>> > > > > > >>>>>>>>> your
>> > > > > > >>>>>>>>>>>>>>>>>>>           approach, I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > would
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would
>> > > remove
>> > > > > any
>> > > > > > >>>>>>>>>>>> misunderstandings
>> > > > > > >>>>>>>>>>>>>>>>>>>           that I may
>> > > > > > >>>>>>>>>>>>>>>>>>>           > have.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking
>> for
>> > my
>> > > > > code.
>> > > > > > I
>> > > > > > >>>>> don't
>> > > > > > >>>>>>> have
>> > > > > > >>>>>>>>>>>>>>>>>>>           something easily
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its
>> bloated
>> > > with
>> > > > > > >>>>> OO-patterns.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K
>> aggKey,
>> > V
>> > > > > > value, T
>> > > > > > >>>>>>>>> aggregate)
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
>> > > > > currentStateAsMap =
>> > > > > > >>>>>>>>>>>> asMap(aggregate);
>> > > > > > >>>>>>>>>>>>>>>>>>> <<
>> > > > > > >>>>>>>>>>>>>>>>>>>           imaginary
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
>> > > > > > >>>>> mapper.apply(value);
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the
>> > > place
>> > > > > > where
>> > > > > > >>>>> people
>> > > > > > >>>>>>>>>>>> actually
>> > > > > > >>>>>>>>>>>>>>>>>>>           gonna have
>> > > > > > >>>>>>>>>>>>>>>>>>>           > issues
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably
>> couldn't
>> > do
>> > > > it.
>> > > > > > we
>> > > > > > >>>>> would
>> > > > > > >>>>>>> need
>> > > > > > >>>>>>>>>>>> to find
>> > > > > > >>>>>>>>>>>>>>>>>>>           a solution
>> > > > > > >>>>>>>>>>>>>>>>>>>           > here.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we
>> propagate
>> > the
>> > > > > > field in
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>>>>> joiner, so
>> > > > > > >>>>>>>>>>>>>>>>>>>           that we can
>> > > > > > >>>>>>>>>>>>>>>>>>>           > pick
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate.
>> > Probably
>> > > > you
>> > > > > > have
>> > > > > > >>>>> not
>> > > > > > >>>>>>>>> thought
>> > > > > > >>>>>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>           this in your
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very
>> open
>> > > to
>> > > > > > find a
>> > > > > > >>>>>>> generic
>> > > > > > >>>>>>>>>>>> solution
>> > > > > > >>>>>>>>>>>>>>>>>>>           here. In my
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is
>> broken
>> > in
>> > > > > > >>>>>>>>> KTableImpl.GroupBy
>> > > > > > >>>>>>>>>>>> that
>> > > > > > >>>>>>>>>>>>>>>>>>> it
>> > > > > > >>>>>>>>>>>>>>>>>>>           looses
>> > > > > > >>>>>>>>>>>>>>>>>>>           > the keys
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the
>> > aggregate
>> > > > > key.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I
>> abstracted
>> > it
>> > > > away
>> > > > > > back
>> > > > > > >>>>>>> then way
>> > > > > > >>>>>>>>>>>> before
>> > > > > > >>>>>>>>>>>>>>>>>>> i
>> > > > > > >>>>>>>>>>>>>>>>>>> was
>> > > > > > >>>>>>>>>>>>>>>>>>>           > thinking
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is
>> > why I
>> > > > > > didn't
>> > > > > > >>>>>>> realize
>> > > > > > >>>>>>>>> its
>> > > > > > >>>>>>>>>>>>>>>>>>>           significance here.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m :
>> current)
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > currentStateAsMap.put(mapper.apply(m),
>> > > > > > >> m);
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > currentStateAsMap.put(toModifyKey,
>> > > > > > >> value);
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > currentStateAsMap.remove(toModifyKey);
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > if(currentStateAsMap.isEmpty()){
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return
>> null;
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
>> > > > > > >>>>>>> asAggregateType(currentStateAsMap)
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at
>> 3:35
>> > PM,
>> > > > Jan
>> > > > > > >>>>> Filipiak
>> > > > > > >>>>>>> <
>> > > > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com
>> <mailto:
>> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing
>> > > Matthias
>> > > > > to
>> > > > > > >>>>> speed!
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I
>> > think
>> > > > > > >> re-keying
>> > > > > > >>>>>>> back
>> > > > > > >>>>>>>>>>>> should be
>> > > > > > >>>>>>>>>>>>>>>>>>>           optional at
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
>> > > > > > KScatteredTable
>> > > > > > >>>>> with
>> > > > > > >>>>>>>>>>>> reshuffle()
>> > > > > > >>>>>>>>>>>>>>>>>>>           returning
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> KTable<originalKey,Joined>
>> > to
>> > > > make
>> > > > > > the
>> > > > > > >>>>>>> backwards
>> > > > > > >>>>>>>>>>>>>>>>>>>           repartitioning
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big
>> favour of
>> > > > doing
>> > > > > > the
>> > > > > > >>>>> out
>> > > > > > >>>>>>> of
>> > > > > > >>>>>>>>> order
>> > > > > > >>>>>>>>>>>>>>>>>>>           processing using
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water
>> mark
>> > > > > tracking.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded
>> > growth
>> > > is
>> > > > > > just
>> > > > > > >>>>> scary
>> > > > > > >>>>>>> + It
>> > > > > > >>>>>>>>>>>> saves
>> > > > > > >>>>>>>>>>>>>>>>>>> us
>> > > > > > >>>>>>>>>>>>>>>>>>>           the header
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction
>> of
>> > > > always
>> > > > > > >>>>>>> repartitioning
>> > > > > > >>>>>>>>>>>> back is
>> > > > > > >>>>>>>>>>>>>>>>>>>           just not so
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has
>> > been
>> > > > > done
>> > > > > > >>>>> before
>> > > > > > >>>>>>> we
>> > > > > > >>>>>>>>>>>> partition
>> > > > > > >>>>>>>>>>>>>>>>>>>           back and
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else
>> afterwards
>> > > is
>> > > > > > really
>> > > > > > >>>>>>> common.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
>> > > > > Bellemare
>> > > > > > >>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your
>> > feedback,
>> > > I
>> > > > do
>> > > > > > >>>>>>> appreciate
>> > > > > > >>>>>>>>> it!
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing
>> would be
>> > > > > > possible,
>> > > > > > >> it
>> > > > > > >>>>>>> would
>> > > > > > >>>>>>>>>>>> require
>> > > > > > >>>>>>>>>>>>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > deserialize
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
>> implies
>> > a
>> > > > > > runtime
>> > > > > > >>>>>>> overhead.
>> > > > > > >>>>>>>>> I
>> > > > > > >>>>>>>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > no
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
>> avoid
>> > > the
>> > > > > > >>>>> overhead.
>> > > > > > >>>>>>> If
>> > > > > > >>>>>>>>> this
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> becomes a
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > problem in
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
>> still
>> > add
>> > > > > name
>> > > > > > >>>>> spacing
>> > > > > > >>>>>>>>> later
>> > > > > > >>>>>>>>>>>> on.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with
>> > > using a
>> > > > > > >> reserved
>> > > > > > >>>>>>> string
>> > > > > > >>>>>>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>>>>           document it.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about
>> the
>> > > > design
>> > > > > it
>> > > > > > >> the
>> > > > > > >>>>>>> type of
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > If
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
>> > > > > correctly,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you
>> have
>> > > > table1
>> > > > > > and
>> > > > > > >>>>> table2
>> > > > > > >>>>>>>>>>>> swapped.
>> > > > > > >>>>>>>>>>>>>>>>>>>           Here is how it
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the
>> records
>> > > that
>> > > > > > contain
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>>>>> foreign key
>> > > > > > >>>>>>>>>>>>>>>>>>>           within their
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
>> > > > > > <a,(fk=A,bar=1)>,
>> > > > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream:
>> <A,X>,
>> > > > <B,Y>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is
>> > required
>> > > > to
>> > > > > > >> extract
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>>>>> foreign
>> > > > > > >>>>>>>>>>>>>>>>>>> key.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key
>> mapper:
>> > (
>> > > > > value
>> > > > > > =>
>> > > > > > >>>>>>> value.fk
>> > > > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to
>> > each
>> > > > > > element
>> > > > > > >> in
>> > > > > > >>>>>>>>> table1,
>> > > > > > >>>>>>>>>>>> and a
>> > > > > > >>>>>>>>>>>>>>>>>>>           new combined
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
>> > > > > (fk=A,bar=1)>,
>> > > > > > >>>>> <A-b,
>> > > > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>> > > > > > >>>>>>>>>>>>>>>>>>>           <B-c,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events
>> are
>> > > > > > >> copartitioned
>> > > > > > >>>>>>> with
>> > > > > > >>>>>>>>>>>> table2:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
>> > > Partition
>> > > > > 0:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
>> <A-a,
>> > > > > > >>>>> (fk=A,bar=1)>,
>> > > > > > >>>>>>> <A-b,
>> > > > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
>> > > Partition
>> > > > > 1:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
>> <B-c,
>> > > > > > >> (fk=B,bar=3)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can
>> be
>> > > > joined
>> > > > > > >>>>> together
>> > > > > > >>>>>>>>> locally
>> > > > > > >>>>>>>>>>>> by
>> > > > > > >>>>>>>>>>>>>>>>>>>           applying the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's
>> design
>> > > and
>> > > > > my
>> > > > > > >>>>> design
>> > > > > > >>>>>>>>>>>> deviate. My
>> > > > > > >>>>>>>>>>>>>>>>>>>           design goes
>> > > > > > >>>>>>>>>>>>>>>>>>>           > on
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
>> > > post-join
>> > > > > and
>> > > > > > >>>>> resolve
>> > > > > > >>>>>>>>>>>> out-of-order
>> > > > > > >>>>>>>>>>>>>>>>>>>           arrival of
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the
>> data
>> > > > keyed
>> > > > > > just
>> > > > > > >>>>> the
>> > > > > > >>>>>>>>>>>> original key.
>> > > > > > >>>>>>>>>>>>>>>>>>>           I do not
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of
>> the
>> > > > > internals
>> > > > > > >>>>>>> outside of
>> > > > > > >>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make
>> > for
>> > > > > larger
>> > > > > > >>>>>>> footprint,
>> > > > > > >>>>>>>>>>>> but it
>> > > > > > >>>>>>>>>>>>>>>>>>>           removes all
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving
>> out-of-order
>> > > > > arrivals
>> > > > > > >> and
>> > > > > > >>>>>>>>> handling
>> > > > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
>> > > > > > >>>>>>>>>>>>>>>>>>>           > the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that
>> this
>> > > makes
>> > > > > the
>> > > > > > >>>>>>> function
>> > > > > > >>>>>>>>> much
>> > > > > > >>>>>>>>>>>>>>>>>>> easier
>> > > > > > >>>>>>>>>>>>>>>>>>>           to use.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this
>> helps
>> > > > resolve
>> > > > > > your
>> > > > > > >>>>>>>>> questions,
>> > > > > > >>>>>>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>>>>           please feel
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on
>> your
>> > > mind.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at
>> 8:36
>> > > PM,
>> > > > > > >>>>> Matthias J.
>> > > > > > >>>>>>>>> Sax <
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io
>> > > <mailto:
>> > > > > > >>>>>>>>>>>> matthias@confluent.io>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up
>> on
>> > > this
>> > > > > > >> thread. I
>> > > > > > >>>>>>> did
>> > > > > > >>>>>>>>> not
>> > > > > > >>>>>>>>>>>> read
>> > > > > > >>>>>>>>>>>>>>>>>>>           everything so
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share
>> couple
>> > of
>> > > > > > initial
>> > > > > > >>>>>>> thoughts:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there
>> is
>> > a
>> > > > > > >> fundamental
>> > > > > > >>>>>>>>>>>> difference
>> > > > > > >>>>>>>>>>>>>>>>>>>           between header
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258.
>> > For
>> > > > 258,
>> > > > > > we
>> > > > > > >> add
>> > > > > > >>>>>>>>> headers
>> > > > > > >>>>>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka
>> Streams
>> > > and
>> > > > > > nobody
>> > > > > > >>>>>>> else is
>> > > > > > >>>>>>>>>>>> supposed
>> > > > > > >>>>>>>>>>>>>>>>>>>           to write
>> > > > > > >>>>>>>>>>>>>>>>>>>           > into
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user
>> > > header
>> > > > > are
>> > > > > > >>>>> written
>> > > > > > >>>>>>> into
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
>> > > conflicts.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't
>> see
>> > a
>> > > > big
>> > > > > > issue
>> > > > > > >>>>> with
>> > > > > > >>>>>>>>> using
>> > > > > > >>>>>>>>>>>>>>>>>>>           headers within
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document
>> it,
>> > > we
>> > > > > can
>> > > > > > >> have
>> > > > > > >>>>>>> some
>> > > > > > >>>>>>>>>>>> "reserved"
>> > > > > > >>>>>>>>>>>>>>>>>>>           header keys
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed
>> to
>> > use
>> > > > > when
>> > > > > > >>>>>>> processing
>> > > > > > >>>>>>>>>>>> data with
>> > > > > > >>>>>>>>>>>>>>>>>>>           Kafka
>> > > > > > >>>>>>>>>>>>>>>>>>>           > Streams.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be
>> ok.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe
>> > way
>> > > to
>> > > > > > avoid
>> > > > > > >>>>>>>>> conflicts,
>> > > > > > >>>>>>>>>>>> since
>> > > > > > >>>>>>>>>>>>>>>>>>> these
>> > > > > > >>>>>>>>>>>>>>>>>>>           > headers
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in
>> internal
>> > > > topics
>> > > > > (I
>> > > > > > >>>>> think):
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and
>> > changelog
>> > > > > > topics,
>> > > > > > >> we
>> > > > > > >>>>> can
>> > > > > > >>>>>>>>>>>> namespace
>> > > > > > >>>>>>>>>>>>>>>>>>>           all headers:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers
>> > are
>> > > > > > >> namespaced
>> > > > > > >>>>> as
>> > > > > > >>>>>>>>>>>> "external."
>> > > > > > >>>>>>>>>>>>>>>>>>> +
>> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
>> > > > > > namespaced as
>> > > > > > >>>>>>>>>>>> "internal." +
>> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing
>> would
>> > be
>> > > > > > >> possible,
>> > > > > > >>>>> it
>> > > > > > >>>>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>>>> require
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
>> implies
>> > a
>> > > > > > runtime
>> > > > > > >>>>>>> overhead.
>> > > > > > >>>>>>>>> I
>> > > > > > >>>>>>>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > no
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
>> avoid
>> > > the
>> > > > > > >>>>> overhead.
>> > > > > > >>>>>>> If
>> > > > > > >>>>>>>>> this
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> becomes a
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > problem in
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can
>> still
>> > add
>> > > > > name
>> > > > > > >>>>> spacing
>> > > > > > >>>>>>>>> later
>> > > > > > >>>>>>>>>>>> on.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about
>> the
>> > > > design
>> > > > > > it
>> > > > > > >> the
>> > > > > > >>>>>>> type
>> > > > > > >>>>>>>>> of
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
>> > > > > correctly,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 =
>> ...
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 =
>> ...
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3>
>> joinedTable
>> > =
>> > > > > > >>>>>>>>>>>> table1.join(table2,...);
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
>> > > `joinedTable`
>> > > > > has
>> > > > > > >> the
>> > > > > > >>>>>>> same
>> > > > > > >>>>>>>>> key
>> > > > > > >>>>>>>>>>>> as the
>> > > > > > >>>>>>>>>>>>>>>>>>>           left input
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not
>> work
>> > > > because
>> > > > > > if
>> > > > > > >>>>> table2
>> > > > > > >>>>>>>>>>>> contains
>> > > > > > >>>>>>>>>>>>>>>>>>>           multiple rows
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in
>> > table1
>> > > > > > (what is
>> > > > > > >>>>> the
>> > > > > > >>>>>>> main
>> > > > > > >>>>>>>>>>>> purpose
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> a
>> > > > > > >>>>>>>>>>>>>>>>>>>           > foreign
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table
>> > > would
>> > > > > only
>> > > > > > >>>>>>> contain a
>> > > > > > >>>>>>>>>>>> single
>> > > > > > >>>>>>>>>>>>>>>>>>>           join result,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > but
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream:
>> <A,X>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
>> > > > <a,(A,1)>,
>> > > > > > >>>>> <b,(A,2)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a
>> > > foreign
>> > > > > key
>> > > > > > to
>> > > > > > >>>>>>> table1
>> > > > > > >>>>>>>>> key
>> > > > > > >>>>>>>>>>>> (ie,
>> > > > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
>> > > > > > >>>>>>>>>>>>>>>>>>>           > If
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same
>> key
>> > > as
>> > > > > key
>> > > > > > of
>> > > > > > >>>>>>> table1,
>> > > > > > >>>>>>>>> this
>> > > > > > >>>>>>>>>>>>>>>>>>>           implies that the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be
>> <A,
>> > > > > > join(X,1)>
>> > > > > > >> or
>> > > > > > >>>>> <A,
>> > > > > > >>>>>>>>>>>> join(X,2)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           but not
>> > > > > > >>>>>>>>>>>>>>>>>>>           > both.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the
>> same
>> > > > key,
>> > > > > > >>>>> whatever
>> > > > > > >>>>>>>>> result
>> > > > > > >>>>>>>>>>>> record
>> > > > > > >>>>>>>>>>>>>>>>>>>           we emit
>> > > > > > >>>>>>>>>>>>>>>>>>>           > later,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous
>> > > result.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why
>> Jan
>> > > > > > originally
>> > > > > > >>>>>>> proposed
>> > > > > > >>>>>>>>>>>> to use
>> > > > > > >>>>>>>>>>>>>>>>>>> a
>> > > > > > >>>>>>>>>>>>>>>>>>>           > combination
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of
>> the
>> > > input
>> > > > > > tables
>> > > > > > >>>>> as
>> > > > > > >>>>>>> key
>> > > > > > >>>>>>>>> of
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           output table.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the
>> > output
>> > > > > table
>> > > > > > >>>>> unique
>> > > > > > >>>>>>> and
>> > > > > > >>>>>>>>> we
>> > > > > > >>>>>>>>>>>> can
>> > > > > > >>>>>>>>>>>>>>>>>>>           store both in
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
>> > > > > join(X,1)>,
>> > > > > > >> <A-b,
>> > > > > > >>>>>>>>>>>> join(X,2)>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan
>> > > > Filipiak
>> > > > > > >> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark
>> could
>> > be
>> > > > > > >>>>> disregarded.
>> > > > > > >>>>>>> The
>> > > > > > >>>>>>>>>>>> decision
>> > > > > > >>>>>>>>>>>>>>>>>>>           about the
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of
>> > the
>> > > > > > >> aggregated
>> > > > > > >>>>>>> map.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long
>> maps
>> > > > would
>> > > > > be
>> > > > > > >>>>>>> unpacked
>> > > > > > >>>>>>>>> and
>> > > > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
>> > > > > > >>>>>>>>>>>>>>>>>>>           > element
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as
>> > > delete.
>> > > > > Any
>> > > > > > >>>>> other
>> > > > > > >>>>>>> count
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in
>> > > "waiting
>> > > > > for
>> > > > > > >>>>> correct
>> > > > > > >>>>>>>>>>>> deletes to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29,
>> Adam
>> > > > > > Bellemare
>> > > > > > >>>>>>> wrote:
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I
>> could
>> > > > > replace
>> > > > > > >> the
>> > > > > > >>>>>>> second
>> > > > > > >>>>>>>>>>>>>>>>>>>           repartition store
>> > > > > > >>>>>>>>>>>>>>>>>>>           > and
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with
>> a
>> > > > groupBy
>> > > > > > and
>> > > > > > >>>>>>> reduce.
>> > > > > > >>>>>>>>>>>> However,
>> > > > > > >>>>>>>>>>>>>>>>>>>           it looks
>> > > > > > >>>>>>>>>>>>>>>>>>>           > like
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store
>> the
>> > > > > > highwater
>> > > > > > >>>>> value
>> > > > > > >>>>>>>>> within
>> > > > > > >>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           materialized
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
>> > > > > > out-of-order
>> > > > > > >>>>>>> records
>> > > > > > >>>>>>>>>>>> (assuming
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>> my
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...).
>> This
>> > > in
>> > > > > > effect
>> > > > > > >> is
>> > > > > > >>>>>>> the
>> > > > > > >>>>>>>>> same
>> > > > > > >>>>>>>>>>>> as
>> > > > > > >>>>>>>>>>>>>>>>>>> the
>> > > > > > >>>>>>>>>>>>>>>>>>>           design I
>> > > > > > >>>>>>>>>>>>>>>>>>>           > have
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two
>> tables
>> > > > merged
>> > > > > > >>>>> together.
>> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> > > > > > >>>>>>>>>>>>>>>>>>>           >
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>> --
>> > > > > > >>>>>>>>>>>>>>>>> -- Guozhang
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>> --
>> > > > > > >>>>>>>>>>>>>>>> -- Guozhang
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>>
>> > > > > > >>>>>>>>>>>
>> > > > > > >>>>>>>>>>
>> > > > > > >>>>>>>>>
>> > > > > > >>>>>>>>
>> > > > > > >>>>>>>
>> > > > > > >>>>>>
>> > > > > > >>>>>
>> > > > > > >>>>
>> > > > > > >>>
>> > > > > > >>
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> >
>> > --
>> > -- Guozhang
>> >
>>
>

-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Guozhang,

Thanks for taking a look! I think Adam's already addressed your questions
as well as I could have.

Hi Adam,

Thanks for updating the KIP. It looks great, especially how all the
need-to-know information is right at the top, followed by the details.

Also, thanks for that high-level diagram. Actually, now that I'm looking at
it, I think part of my proposal got lost in translation, although I do
think that what you have there is also correct.

I sketched up a crude diagram based on yours and attached it to the KIP
(I'm not sure if attached or inline images work on the mailing list):
https://cwiki.apache.org/confluence/download/attachments/74684836/suggestion.png
. It's also attached to this email for convenience.

Hopefully, you can see how it's intended to line up, and which parts are
modified.
At a high level, instead of performing the join on the right-hand side,
we're essentially just registering interest, like "LHS key A wishes to
receive updates for RHS key 2". Then, when there is a new "interest" or any
updates to the RHS records, it "broadcasts" its state back to the LHS
records who are interested in it.

Thus, instead of sending the LHS values to the RHS joiner workers and then
sending the join results back to the LHS worke be co-partitioned and
validated, we instead only send the LHS *keys* to the RHS workers and then
only the RHS k/v back to be joined by the LHS worker.

I've been considering both your diagram and mine, and I *think* what I'm
proposing has a few advantages.

Here are some points of interest as you look at the diagram:
* When we extract the foreign key and send it to the Pre-Join Repartition
Topic, we can send only the FK/PK pair. There's no need to worry about
custom partitioner logic, since we can just use the foreign key plainly as
the repartition record key. Also, we save on transmitting the LHS value,
since we only send its key in this step.
* We also only need to store the RHSKey:LHSKey mapping in the
MaterializedSubscriptionStore, saving on disk. We can use the same rocks
key format you proposed and the same algorithm involving range scans when
the RHS records get updated.
* Instead of joining on the right side, all we do is compose a
re-repartition record so we can broadcast the RHS k/v pair back to the
original LHS partition. (this is what the "rekey" node is doing)
* Then, there is a special kind of Joiner that's co-resident in the same
StreamTask as the LHS table, subscribed to the Post-Join Repartition Topic.
** This Joiner is *not* triggered directly by any changes in the LHS
KTable. Instead, LHS events indirectly trigger the join via the whole
lifecycle.
** For each event arriving from the Post-Join Repartition Topic, the Joiner
looks up the corresponding record in the LHS KTable. It validates the FK as
you noted, discarding any inconsistent events. Otherwise, it unpacks the
RHS K/V pair and invokes the ValueJoiner to obtain the join result
** Note that the Joiner itself is stateless, so materializing the join
result is optional, just as with the 1:1 joins.

So in summary:
* instead of transmitting the LHS keys and values to the right and the
JoinResult back to the left, we only transmit the LHS keys to the right and
the RHS values to the left. Assuming the average RHS value is on smaller
than or equal to the average join result size, it's a clear win on broker
traffic. I think this is actually a reasonable assumption, which we can
discuss more if you're suspicious.
* we only need one copy of the data (the left and right tables need to be
materialized) and one extra copy of the PK:FK pairs in the Materialized
Subscription Store. Materializing the join result is optional, just as with
the existing 1:1 joins.
* we still need the fancy range-scan algorithm on the right to locate all
interested LHS keys when a RHS value is updated, but we don't need a custom
partitioner for either repartition topic (this is of course a modification
we could make to your version as well)

How does this sound to you? (And did I miss anything?)
-John

On Mon, Dec 17, 2018 at 9:00 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi John & Guozhang
>
> @John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the KIP,
> pruned much of what I wrote and put a simplified diagram near the top to
> illustrate the workflow. I encapsulated Jan's content at the bottom of the
> document. I believe it is simpler to read by far now.
>
> @Guozhang Wang <wa...@gmail.com>:
> > #1: rekey left table
> >   -> source from the left upstream, send to rekey-processor to generate
> combined key, and then sink to copartition topic.
> Correct.
>
> > #2: first-join with right table
> >   -> source from the right table upstream, materialize the right table.
> >   -> source from the co-partition topic, materialize the rekeyed left
> table, join with the right table, rekey back, and then sink to the
> rekeyed-back topic.
> Almost - I cleared up the KIP. We do not rekey back yet, as I need the
> Foreign-Key value generated in #1 above to compare in the resolution stage.
>
> > #3: second join
> >    -> source from the rekeyed-back topic, materialize the rekeyed back
> table.
> >   -> source from the left upstream, materialize the left table, join with
> the rekeyed back table.
> Almost - As each event comes in, we just run it through a stateful
> processor that checks the original ("This") KTable for the key. The value
> payload then has the foreignKeyExtractor applied again as in Part #1 above,
> and gets the current foreign key. Then we compare it to the joined event
> that we are currently resolving. If they have the same foreign-key,
> propagate the result out. If they don't, throw the event away.
>
> The end result is that we do need to materialize 2 additional tables
> (left/this-combinedkey table, and the final Joined table) as I've
> illustrated in the updated KIP. I hope the diagram clears it up a lot
> better. Please let me know.
>
> Thanks again
> Adam
>
>
>
>
> On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > John,
> >
> > Thanks a lot for the suggestions on refactoring the wiki, I agree with
> you
> > that we should consider the KIP proposal to be easily understood by
> anyone
> > in the future to read, and hence should provide a good summary on the
> > user-facing interfaces, as well as rejected alternatives to represent
> > briefly "how we came a long way to this conclusion, and what we have
> > argued, disagreed, and agreed about, etc" so that readers do not need to
> > dig into the DISCUSS thread to get all the details. We can, of course,
> keep
> > the implementation details like "workflows" on the wiki page as a
> addendum
> > section since it also has correlations.
> >
> > Regarding your proposal on comment 6): that's a very interesting idea!
> Just
> > to clarify that I understands it fully correctly: the proposal's resulted
> > topology is still the same as the current proposal, where we will have 3
> > sub-topologies for this operator:
> >
> > #1: rekey left table
> >    -> source from the left upstream, send to rekey-processor to generate
> > combined key, and then sink to copartition topic.
> >
> > #2: first-join with right table
> >    -> source from the right table upstream, materialize the right table.
> >    -> source from the co-partition topic, materialize the rekeyed left
> > table, join with the right table, rekey back, and then sink to the
> > rekeyed-back topic.
> >
> > #3: second join
> >    -> source from the rekeyed-back topic, materialize the rekeyed back
> > table.
> >    -> source from the left upstream, materialize the left table, join
> with
> > the rekeyed back table.
> >
> > Sub-topology #1 and #3 may be merged to a single sub-topology since both
> of
> > them read from the left table source stream. In this workflow, we need to
> > materialize 4 tables (left table in #3, right table in #2, rekeyed left
> > table in #2, rekeyed-back table in #3), and 2 repartition topics
> > (copartition topic, rekeyed-back topic).
> >
> > Compared with Adam's current proposal in the workflow overview, it has
> the
> > same num.materialize tables (left table, rekeyed left table, right table,
> > out-of-ordering resolver table), and same num.internal topics (two). The
> > advantage is that on the copartition topic, we can save bandwidth by not
> > sending value, and in #2 the rekeyed left table is smaller since we do
> not
> > have any values to materialize. Is that right?
> >
> >
> > Guozhang
> >
> >
> >
> > On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io> wrote:
> >
> > > Hi Adam,
> > >
> > > Given that the committers are all pretty busy right now, I think that
> it
> > > would help if you were to refactor the KIP a little to reduce the
> > workload
> > > for reviewers.
> > >
> > > I'd recommend the following changes:
> > > * relocate all internal details to a section at the end called
> something
> > > like "Implementation Notes" or something like that.
> > > * rewrite the rest of the KIP to be a succinct as possible and mention
> > only
> > > publicly-facing API changes.
> > > ** for example, the interface that you've already listed there, as well
> > as
> > > a textual description of the guarantees we'll be providing (join result
> > is
> > > copartitioned with the LHS, and the join result is guaranteed correct)
> > >
> > > A good target would be that the whole main body of the KIP, including
> > > Status, Motivation, Proposal, Justification, and Rejected Alternatives
> > all
> > > fit "above the fold" (i.e., all fit on the screen at a comfortable zoom
> > > level).
> > > I think the only real Rejected Alternative that bears mention at this
> > point
> > > is KScatteredTable, which you could just include the executive summary
> on
> > > (no implementation details), and link to extra details in the
> > > Implementation Notes section.
> > >
> > > Taking a look at the wiki page, ~90% of the text there is internal
> > detail,
> > > which is useful for the dubious, but doesn't need to be ratified in a
> > vote
> > > (and would be subject to change without notice in the future anyway).
> > > There's also a lot of conflicting discussion, as you've very
> respectfully
> > > tried to preserve the original proposal from Jan while adding your own.
> > > Isolating all this information in a dedicated section at the bottom
> frees
> > > the voters up to focus on the public API part of the proposal, which is
> > > really all they need to consider.
> > >
> > > Plus, it'll be clear to future readers which parts of the document are
> > > enduring, and which parts are a snapshot of our implementation thinking
> > at
> > > the time.
> > >
> > > I'm suggesting this because I suspect that the others haven't made time
> > to
> > > review it partly because it seems daunting. If it seems like it would
> be
> > a
> > > huge time investment to review, people will just keep putting it off.
> But
> > > if the KIP is a single page, then they'll be more inclined to give it a
> > > read.
> > >
> > > Honestly, I don't think the KIP itself is that controversial (apart
> from
> > > the scattered table thing (sorry, Jan) ). Most of the discussion has
> been
> > > around the implementation, which we can continue more effectively in a
> PR
> > > once the KIP has passed.
> > >
> > > How does that sound?
> > > -John
> > >
> > > On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <
> adam.bellemare@gmail.com
> > >
> > > wrote:
> > >
> > > > 1) I believe that the resolution mechanism John has proposed is
> > > sufficient
> > > > - it is clean and easy and doesn't require additional RocksDB stores,
> > > which
> > > > reduces the footprint greatly. I don't think we need to resolve based
> > on
> > > > timestamp or offset anymore, but if we decide to do to that would be
> > > within
> > > > the bounds of the existing API.
> > > >
> > > > 2) Is the current API sufficient, or does it need to be altered to go
> > > back
> > > > to vote?
> > > >
> > > > 3) KScatteredTable implementation can always be added in a future
> > > revision.
> > > > This API does not rule it out. This implementation of this function
> > would
> > > > simply be replaced with `KScatteredTable.resolve()` while still
> > > maintaining
> > > > the existing API, thereby giving both features as Jan outlined
> earlier.
> > > > Would this work?
> > > >
> > > >
> > > > Thanks Guozhang, John and Jan
> > > >
> > > >
> > > >
> > > >
> > > > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io>
> > wrote:
> > > >
> > > > > Hi, all,
> > > > >
> > > > > >> In fact, we
> > > > > >> can just keep a single final-result store with timestamps and
> > reject
> > > > > values
> > > > > >> that have a smaller timestamp, is that right?
> > > > >
> > > > > > Which is the correct output should at least be decided on the
> > offset
> > > of
> > > > > > the original message.
> > > > >
> > > > > Thanks for this point, Jan.
> > > > >
> > > > > KIP-258 is merely to allow embedding the record timestamp  in the
> k/v
> > > > > store,
> > > > > as well as providing a storage-format upgrade path.
> > > > >
> > > > > I might have missed it, but I think we have yet to discuss whether
> > it's
> > > > > safe
> > > > > or desirable just to swap topic-ordering our for
> timestamp-ordering.
> > > This
> > > > > is
> > > > > a very deep topic, and I think it would only pollute the current
> > > > > discussion.
> > > > >
> > > > > What Adam has proposed is safe, given the *current* ordering
> > semantics
> > > > > of the system. If we can agree on his proposal, I think we can
> merge
> > > the
> > > > > feature well before the conversation about timestamp ordering even
> > > takes
> > > > > place, much less reaches a conclusion. In the mean time, it would
> > seem
> > > to
> > > > > be unfortunate to have one join operator with different ordering
> > > > semantics
> > > > > from every other KTable operator.
> > > > >
> > > > > If and when that timestamp discussion takes place, many (all?)
> KTable
> > > > > operations
> > > > > will need to be updated, rendering the many:one join a small
> marginal
> > > > cost.
> > > > >
> > > > > And, just to plug it again, I proposed an algorithm above that I
> > > believe
> > > > > provides
> > > > > correct ordering without any additional metadata, and regardless of
> > the
> > > > > ordering semantics. I didn't bring it up further, because I felt
> the
> > > KIP
> > > > > only needs
> > > > > to agree on the public API, and we can discuss the implementation
> at
> > > > > leisure in
> > > > > a PR...
> > > > >
> > > > > Thanks,
> > > > > -John
> > > > >
> > > > >
> > > > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> > Jan.Filipiak@trivago.com
> > > >
> > > > > wrote:
> > > > >
> > > > > >
> > > > > >
> > > > > > On 10.12.2018 07:42, Guozhang Wang wrote:
> > > > > > > Hello Adam / Jan / John,
> > > > > > >
> > > > > > > Sorry for being late on this thread! I've finally got some time
> > > this
> > > > > > > weekend to cleanup a load of tasks on my queue (actually I've
> > also
> > > > > > realized
> > > > > > > there are a bunch of other things I need to enqueue while
> > cleaning
> > > > them
> > > > > > up
> > > > > > > --- sth I need to improve on my side). So here are my thoughts:
> > > > > > >
> > > > > > > Regarding the APIs: I like the current written API in the KIP.
> > More
> > > > > > > generally I'd prefer to keep the 1) one-to-many join
> > > functionalities
> > > > as
> > > > > > > well as 2) other join types than inner as separate KIPs since
> 1)
> > > may
> > > > > > worth
> > > > > > > a general API refactoring that can benefit not only foreignkey
> > > joins
> > > > > but
> > > > > > > collocate joins as well (e.g. an extended proposal of
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > > > ),
> > > > > > > and I'm not sure if other join types would actually be needed
> > > (maybe
> > > > > left
> > > > > > > join still makes sense), so it's better to
> > > > > wait-for-people-to-ask-and-add
> > > > > > > than add-sth-that-no-one-uses.
> > > > > > >
> > > > > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > > > > > > KScatteredTable for users to inject their own optimization: I'd
> > > > prefer
> > > > > to
> > > > > > > do the current option as-is, and my main rationale is for
> > > > optimization
> > > > > > > rooms inside the Streams internals and the API succinctness.
> For
> > > > > advanced
> > > > > > > users who may indeed prefer KScatteredTable and do their own
> > > > > > optimization,
> > > > > > > while it is too much of the work to use Processor API
> directly, I
> > > > think
> > > > > > we
> > > > > > > can still extend the current API to support it in the future if
> > it
> > > > > > becomes
> > > > > > > necessary.
> > > > > >
> > > > > > no internal optimization potential. it's a myth
> > > > > >
> > > > > > ¯\_(ツ)_/¯
> > > > > >
> > > > > > :-)
> > > > > >
> > > > > > >
> > > > > > > Another note about step 4) resolving out-of-ordering data, as I
> > > > > mentioned
> > > > > > > before I think with KIP-258 (embedded timestamp with key-value
> > > store)
> > > > > we
> > > > > > > can actually make this step simpler than the current proposal.
> In
> > > > fact,
> > > > > > we
> > > > > > > can just keep a single final-result store with timestamps and
> > > reject
> > > > > > values
> > > > > > > that have a smaller timestamp, is that right?
> > > > > >
> > > > > > Which is the correct output should at least be decided on the
> > offset
> > > of
> > > > > > the original message.
> > > > > >
> > > > > > >
> > > > > > >
> > > > > > > That's all I have in mind now. Again, great appreciation to
> Adam
> > to
> > > > > make
> > > > > > > such HUGE progress on this KIP!
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> > > > Jan.Filipiak@trivago.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> If they don't find the time:
> > > > > > >> They usually take the opposite path from me :D
> > > > > > >> so the answer would be clear.
> > > > > > >>
> > > > > > >> hence my suggestion to vote.
> > > > > > >>
> > > > > > >>
> > > > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> > > > > > >>> Hi Guozhang and Matthias
> > > > > > >>>
> > > > > > >>> I know both of you are quite busy, but we've gotten this KIP
> > to a
> > > > > point
> > > > > > >>> where we need more guidance on the API (perhaps a bit of a
> > > > > tie-breaker,
> > > > > > >> if
> > > > > > >>> you will). If you have anyone else you may think should look
> at
> > > > this,
> > > > > > >>> please tag them accordingly.
> > > > > > >>>
> > > > > > >>> The scenario is as such:
> > > > > > >>>
> > > > > > >>> Current Option:
> > > > > > >>> API:
> > > > > > >>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > > > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
> > > partition
> > > > > with
> > > > > > >> the
> > > > > > >>> foreignKey (repartition 1)
> > > > > > >>> 2) Join the data
> > > > > > >>> 3) Shuffle the data back to the original node (repartition 2)
> > > > > > >>> 4) Resolve out-of-order arrival / race condition due to
> > > foreign-key
> > > > > > >> changes.
> > > > > > >>>
> > > > > > >>> Alternate Option:
> > > > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
> > > > > > >>> - It would be keyed on a wrapped key function:
> <CombinedKey<KO,
> > > K>,
> > > > > VR>
> > > > > > >> (KO
> > > > > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> > > > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
> > > otherwise a
> > > > > > user
> > > > > > >>> would be able to perform additional functions directly from
> the
> > > > > > >>> KScatteredTable (TBD - currently out of scope).
> > > > > > >>> - John's analysis 2-emails up is accurate as to the
> tradeoffs.
> > > > > > >>>
> > > > > > >>> Current Option is coded as-is. Alternate option is possible,
> > but
> > > > will
> > > > > > >>> require for implementation details to be made in the API and
> > some
> > > > > > >> exposure
> > > > > > >>> of new data structures into the API (ie: CombinedKey).
> > > > > > >>>
> > > > > > >>> I appreciate any insight into this.
> > > > > > >>>
> > > > > > >>> Thanks.
> > > > > > >>>
> > > > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > > > > > adam.bellemare@gmail.com>
> > > > > > >>> wrote:
> > > > > > >>>
> > > > > > >>>> Hi John
> > > > > > >>>>
> > > > > > >>>> Thanks for your feedback and assistance. I think your
> summary
> > is
> > > > > > >> accurate
> > > > > > >>>> from my perspective. Additionally, I would like to add that
> > > there
> > > > > is a
> > > > > > >> risk
> > > > > > >>>> of inconsistent final states without performing the
> > resolution.
> > > > This
> > > > > > is
> > > > > > >> a
> > > > > > >>>> major concern for me as most of the data I have dealt with
> is
> > > > > produced
> > > > > > >> by
> > > > > > >>>> relational databases. We have seen a number of cases where a
> > > user
> > > > in
> > > > > > the
> > > > > > >>>> Rails UI has modified the field (foreign key), realized they
> > > made
> > > > a
> > > > > > >>>> mistake, and then updated the field again with a new key.
> The
> > > > events
> > > > > > are
> > > > > > >>>> propagated out as they are produced, and as such we have had
> > > > > > real-world
> > > > > > >>>> cases where these inconsistencies were propagated downstream
> > as
> > > > the
> > > > > > >> final
> > > > > > >>>> values due to the race conditions in the fanout of the data.
> > > > > > >>>>
> > > > > > >>>> This solution that I propose values correctness of the final
> > > > result
> > > > > > over
> > > > > > >>>> other factors.
> > > > > > >>>>
> > > > > > >>>> We could always move this function over to using a
> > > KScatteredTable
> > > > > > >>>> implementation in the future, and simply deprecate it this
> > join
> > > > API
> > > > > in
> > > > > > >>>> time. I think I would like to hear more from some of the
> other
> > > > major
> > > > > > >>>> committers on which course of action they would think is
> best
> > > > before
> > > > > > any
> > > > > > >>>> more coding is done.
> > > > > > >>>>
> > > > > > >>>> Thanks again
> > > > > > >>>>
> > > > > > >>>> Adam
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> > john@confluent.io>
> > > > > > wrote:
> > > > > > >>>>
> > > > > > >>>>> Hi Jan and Adam,
> > > > > > >>>>>
> > > > > > >>>>> Wow, thanks for doing that test, Adam. Those results are
> > > > > encouraging.
> > > > > > >>>>>
> > > > > > >>>>> Thanks for your performance experience as well, Jan. I
> agree
> > > that
> > > > > > >> avoiding
> > > > > > >>>>> unnecessary join outputs is especially important when the
> > > fan-out
> > > > > is
> > > > > > so
> > > > > > >>>>> high. I suppose this could also be built into the
> > > implementation
> > > > > > we're
> > > > > > >>>>> discussing, but it wouldn't have to be specified in the KIP
> > > > (since
> > > > > > >> it's an
> > > > > > >>>>> API-transparent optimization).
> > > > > > >>>>>
> > > > > > >>>>> As far as whether or not to re-repartition the data, I
> didn't
> > > > bring
> > > > > > it
> > > > > > >> up
> > > > > > >>>>> because it sounded like the two of you agreed to leave the
> > KIP
> > > > > as-is,
> > > > > > >>>>> despite the disagreement.
> > > > > > >>>>>
> > > > > > >>>>> If you want my opinion, I feel like both approaches are
> > > > reasonable.
> > > > > > >>>>> It sounds like Jan values more the potential for developers
> > to
> > > > > > optimize
> > > > > > >>>>> their topologies to re-use the intermediate nodes, whereas
> > Adam
> > > > > > places
> > > > > > >>>>> more
> > > > > > >>>>> value on having a single operator that people can use
> without
> > > > extra
> > > > > > >> steps
> > > > > > >>>>> at the end.
> > > > > > >>>>>
> > > > > > >>>>> Personally, although I do find it exceptionally annoying
> > when a
> > > > > > >> framework
> > > > > > >>>>> gets in my way when I'm trying to optimize something, it
> > seems
> > > > > better
> > > > > > >> to
> > > > > > >>>>> go
> > > > > > >>>>> for a single operation.
> > > > > > >>>>> * Encapsulating the internal transitions gives us
> significant
> > > > > > latitude
> > > > > > >> in
> > > > > > >>>>> the implementation (for example, joining only at the end,
> not
> > > in
> > > > > the
> > > > > > >>>>> middle
> > > > > > >>>>> to avoid extra data copying and out-of-order resolution;
> how
> > we
> > > > > > >> represent
> > > > > > >>>>> the first repartition keys (combined keys vs. value
> vectors),
> > > > > etc.).
> > > > > > >> If we
> > > > > > >>>>> publish something like a KScatteredTable with the
> > > > right-partitioned
> > > > > > >> joined
> > > > > > >>>>> data, then the API pretty much locks in the implementation
> as
> > > > well.
> > > > > > >>>>> * The API seems simpler to understand and use. I do mean
> > > "seems";
> > > > > if
> > > > > > >>>>> anyone
> > > > > > >>>>> wants to make the case that KScatteredTable is actually
> > > simpler,
> > > > I
> > > > > > >> think
> > > > > > >>>>> hypothetical usage code would help. From a relational
> algebra
> > > > > > >> perspective,
> > > > > > >>>>> it seems like KTable.join(KTable) should produce a new
> KTable
> > > in
> > > > > all
> > > > > > >>>>> cases.
> > > > > > >>>>> * That said, there might still be room in the API for a
> > > different
> > > > > > >>>>> operation
> > > > > > >>>>> like what Jan has proposed to scatter a KTable, and then do
> > > > things
> > > > > > like
> > > > > > >>>>> join, re-group, etc from there... I'm not sure; I haven't
> > > thought
> > > > > > >> through
> > > > > > >>>>> all the consequences yet.
> > > > > > >>>>>
> > > > > > >>>>> This is all just my opinion after thinking over the
> > discussion
> > > so
> > > > > > >> far...
> > > > > > >>>>> -John
> > > > > > >>>>>
> > > > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > > > > > >> adam.bellemare@gmail.com>
> > > > > > >>>>> wrote:
> > > > > > >>>>>
> > > > > > >>>>>> Updated the PR to take into account John's feedback.
> > > > > > >>>>>>
> > > > > > >>>>>> I did some preliminary testing for the performance of the
> > > > > > prefixScan.
> > > > > > >> I
> > > > > > >>>>>> have attached the file, but I will also include the text
> in
> > > the
> > > > > body
> > > > > > >>>>> here
> > > > > > >>>>>> for archival purposes (I am not sure what happens to
> > attached
> > > > > > files).
> > > > > > >> I
> > > > > > >>>>>> also updated the PR and the KIP accordingly.
> > > > > > >>>>>>
> > > > > > >>>>>> Summary: It scales exceptionally well for scanning large
> > > values
> > > > of
> > > > > > >>>>>> records. As Jan mentioned previously, the real issue would
> > be
> > > > more
> > > > > > >>>>> around
> > > > > > >>>>>> processing the resulting records after obtaining them. For
> > > > > instance,
> > > > > > >> it
> > > > > > >>>>>> takes approximately ~80-120 mS to flush the buffer and a
> > > further
> > > > > > >>>>> ~35-85mS
> > > > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them.
> > > > > Iterating
> > > > > > >>>>>> through the records just to generate a simple count takes
> ~
> > 40
> > > > > times
> > > > > > >>>>> longer
> > > > > > >>>>>> than the flush + scan combined.
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > > >>>>>> Setup:
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
> > > (Xmx512m,
> > > > > > >> Xms512m)
> > > > > > >>>>>> CPU: i7 2.2 Ghz.
> > > > > > >>>>>>
> > > > > > >>>>>> Note: I am using a slightly-modified, directly-accessible
> > > Kafka
> > > > > > >> Streams
> > > > > > >>>>>> RocksDB
> > > > > > >>>>>> implementation (RocksDB.java, basically just avoiding the
> > > > > > >>>>>> ProcessorContext).
> > > > > > >>>>>> There are no modifications to the default RocksDB values
> > > > provided
> > > > > in
> > > > > > >> the
> > > > > > >>>>>> 2.1/trunk release.
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>> keysize = 128 bytes
> > > > > > >>>>>> valsize = 512 bytes
> > > > > > >>>>>>
> > > > > > >>>>>> Step 1:
> > > > > > >>>>>> Write X positive matching events: (key = prefix +
> > left-padded
> > > > > > >>>>>> auto-incrementing integer)
> > > > > > >>>>>> Step 2:
> > > > > > >>>>>> Write 10X negative matching events (key = left-padded
> > > > > > >> auto-incrementing
> > > > > > >>>>>> integer)
> > > > > > >>>>>> Step 3:
> > > > > > >>>>>> Perform flush
> > > > > > >>>>>> Step 4:
> > > > > > >>>>>> Perform prefixScan
> > > > > > >>>>>> Step 5:
> > > > > > >>>>>> Iterate through return Iterator and validate the count of
> > > > expected
> > > > > > >>>>> events.
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > > >>>>>> Results:
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > > >>>>>> X = 1k (11k events total)
> > > > > > >>>>>> Flush Time = 39 mS
> > > > > > >>>>>> Scan Time = 7 mS
> > > > > > >>>>>> 6.9 MB disk
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > > >>>>>> X = 10k (110k events total)
> > > > > > >>>>>> Flush Time = 45 mS
> > > > > > >>>>>> Scan Time = 8 mS
> > > > > > >>>>>> 127 MB
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > > >>>>>> X = 100k (1.1M events total)
> > > > > > >>>>>> Test1:
> > > > > > >>>>>> Flush Time = 60 mS
> > > > > > >>>>>> Scan Time = 12 mS
> > > > > > >>>>>> 678 MB
> > > > > > >>>>>>
> > > > > > >>>>>> Test2:
> > > > > > >>>>>> Flush Time = 45 mS
> > > > > > >>>>>> Scan Time = 7 mS
> > > > > > >>>>>> 576 MB
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > > >>>>>> X = 1MB (11M events total)
> > > > > > >>>>>> Test1:
> > > > > > >>>>>> Flush Time = 52 mS
> > > > > > >>>>>> Scan Time = 19 mS
> > > > > > >>>>>> 7.2 GB
> > > > > > >>>>>>
> > > > > > >>>>>> Test2:
> > > > > > >>>>>> Flush Time = 84 mS
> > > > > > >>>>>> Scan Time = 34 mS
> > > > > > >>>>>> 9.1 GB
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > > >>>>>> X = 2.5M (27.5M events total)
> > > > > > >>>>>> Test1:
> > > > > > >>>>>> Flush Time = 82 mS
> > > > > > >>>>>> Scan Time = 63 mS
> > > > > > >>>>>> 17GB - 276 sst files
> > > > > > >>>>>>
> > > > > > >>>>>> Test2:
> > > > > > >>>>>> Flush Time = 116 mS
> > > > > > >>>>>> Scan Time = 35 mS
> > > > > > >>>>>> 23GB - 361 sst files
> > > > > > >>>>>>
> > > > > > >>>>>> Test3:
> > > > > > >>>>>> Flush Time = 103 mS
> > > > > > >>>>>> Scan Time = 82 mS
> > > > > > >>>>>> 19 GB - 300 sst files
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > > >>>>>>
> > > > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M
> events. I
> > > > tried
> > > > > > to
> > > > > > >> go
> > > > > > >>>>>> to X = 10M (110M events) but RocksDB was going into the
> > 100GB+
> > > > > range
> > > > > > >>>>> and my
> > > > > > >>>>>> laptop ran out of disk. More extensive testing could be
> done
> > > > but I
> > > > > > >>>>> suspect
> > > > > > >>>>>> that it would be in line with what we're seeing in the
> > results
> > > > > > above.
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>> At this point in time, I think the only major discussion
> > point
> > > > is
> > > > > > >> really
> > > > > > >>>>>> around what Jan and I have disagreed on: repartitioning
> > back +
> > > > > > >> resolving
> > > > > > >>>>>> potential out of order issues or leaving that up to the
> > client
> > > > to
> > > > > > >>>>> handle.
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>> Thanks folks,
> > > > > > >>>>>>
> > > > > > >>>>>> Adam
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > > > > > Jan.Filipiak@trivago.com
> > > > > > >>>
> > > > > > >>>>>> wrote:
> > > > > > >>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > > > > > >>>>>>>> Hi all,
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Sorry that this discussion petered out... I think the
> 2.1
> > > > > release
> > > > > > >>>>>>> caused an
> > > > > > >>>>>>>> extended distraction that pushed it off everyone's radar
> > > > (which
> > > > > > was
> > > > > > >>>>>>>> precisely Adam's concern). Personally, I've also had
> some
> > > > extend
> > > > > > >>>>>>>> distractions of my own that kept (and continue to keep)
> me
> > > > > > >>>>> preoccupied.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> However, calling for a vote did wake me up, so I guess
> Jan
> > > was
> > > > > on
> > > > > > >> the
> > > > > > >>>>>>> right
> > > > > > >>>>>>>> track!
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> I've gone back and reviewed the whole KIP document and
> the
> > > > prior
> > > > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> API Thoughts:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 1. If I read the KIP right, you are proposing a
> > many-to-one
> > > > > join.
> > > > > > >>>>> Could
> > > > > > >>>>>>> we
> > > > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer,
> flip
> > > the
> > > > > > design
> > > > > > >>>>>>> around
> > > > > > >>>>>>>> and make it a oneToManyJoin?
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join
> > > type,
> > > > > and
> > > > > > it
> > > > > > >>>>>>> seems
> > > > > > >>>>>>>> like it might trick some people into using it for a
> > > one-to-one
> > > > > > join.
> > > > > > >>>>>>> This
> > > > > > >>>>>>>> would work, of course, but it would be super inefficient
> > > > > compared
> > > > > > to
> > > > > > >>>>> a
> > > > > > >>>>>>>> simple rekey-and-join.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 2. I might have missed it, but I don't think it's
> > specified
> > > > > > whether
> > > > > > >>>>>>> it's an
> > > > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer join,
> as
> > > > > > >>>>> (neglecting
> > > > > > >>>>>>> IQ),
> > > > > > >>>>>>>> the rest can be achieved by filtering or by handling it
> in
> > > the
> > > > > > >>>>>>> ValueJoiner.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite
> > > right.
> > > > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
> > > paradigms
> > > > in
> > > > > > >>>>> play in
> > > > > > >>>>>>>> the Streams API, so it's confusing, but instead of three
> > > > > > Serialized
> > > > > > >>>>>>> args, I
> > > > > > >>>>>>>> think it would be better to have one that allows
> > > (optionally)
> > > > > > >> setting
> > > > > > >>>>>>> the 4
> > > > > > >>>>>>>> incoming serdes. The result serde is defined by the
> > > > > Materialized.
> > > > > > >> The
> > > > > > >>>>>>>> incoming serdes can be optional because they might
> already
> > > be
> > > > > > >>>>> available
> > > > > > >>>>>>> on
> > > > > > >>>>>>>> the source KTables, or the default serdes from the
> config
> > > > might
> > > > > be
> > > > > > >>>>>>>> applicable.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins
> > > don't
> > > > > > allow
> > > > > > >>>>>>> setting
> > > > > > >>>>>>>> one, and it seems like it might actually be harmful,
> since
> > > the
> > > > > > rekey
> > > > > > >>>>>>>> operation needs to produce results that are
> co-partitioned
> > > > with
> > > > > > the
> > > > > > >>>>>>> "other"
> > > > > > >>>>>>>> KTable.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I
> didn't
> > > > > actually
> > > > > > >>>>>>> follow
> > > > > > >>>>>>>> what Matthias meant about namespacing requiring
> > > > "deserializing"
> > > > > > the
> > > > > > >>>>>>> record
> > > > > > >>>>>>>> header. The headers are already Strings, so I don't
> think
> > > that
> > > > > > >>>>>>>> deserialization is required. If we applied the namespace
> > at
> > > > > source
> > > > > > >>>>> nodes
> > > > > > >>>>>>>> and stripped it at sink nodes, this would be practically
> > no
> > > > > > >> overhead.
> > > > > > >>>>>>> The
> > > > > > >>>>>>>> advantage of the namespace idea is that no public API
> > change
> > > > wrt
> > > > > > >>>>> headers
> > > > > > >>>>>>>> needs to happen, and no restrictions need to be placed
> on
> > > > users'
> > > > > > >>>>>>> headers.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> (Although I'm wondering if we can get away without the
> > > header
> > > > at
> > > > > > >>>>> all...
> > > > > > >>>>>>>> stay tuned)
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM
> table
> > > > > growing
> > > > > > >>>>>>> without
> > > > > > >>>>>>>> bound. As I read it, the HWM table is effectively
> > > implementing
> > > > > OCC
> > > > > > >> to
> > > > > > >>>>>>>> resolve the problem you noted with disordering when the
> > > rekey
> > > > is
> > > > > > >>>>>>>> reversed... particularly notable when the FK changes. As
> > > such,
> > > > > it
> > > > > > >>>>> only
> > > > > > >>>>>>>> needs to track the most recent "version" (the offset in
> > the
> > > > > source
> > > > > > >>>>>>>> partition) of each key. Therefore, it should have the
> same
> > > > > number
> > > > > > of
> > > > > > >>>>>>> keys
> > > > > > >>>>>>>> as the source table at all times.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> I see that you are aware of KIP-258, which I think might
> > be
> > > > > > relevant
> > > > > > >>>>> in
> > > > > > >>>>>>> a
> > > > > > >>>>>>>> couple of ways. One: it's just about storing the
> timestamp
> > > in
> > > > > the
> > > > > > >>>>> state
> > > > > > >>>>>>>> store, but the ultimate idea is to effectively use the
> > > > timestamp
> > > > > > as
> > > > > > >>>>> an
> > > > > > >>>>>>> OCC
> > > > > > >>>>>>>> "version" to drop disordered updates. You wouldn't want
> to
> > > use
> > > > > the
> > > > > > >>>>>>>> timestamp for this operation, but if you were to use a
> > > similar
> > > > > > >>>>>>> mechanism to
> > > > > > >>>>>>>> store the source offset in the store alongside the
> > re-keyed
> > > > > > values,
> > > > > > >>>>> then
> > > > > > >>>>>>>> you could avoid a separate table.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 6. You and Jan have been thinking about this for a long
> > > time,
> > > > so
> > > > > > >> I've
> > > > > > >>>>>>>> probably missed something here, but I'm wondering if we
> > can
> > > > > avoid
> > > > > > >> the
> > > > > > >>>>>>> HWM
> > > > > > >>>>>>>> tracking at all and resolve out-of-order during a final
> > join
> > > > > > >>>>> instead...
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Let's say we're joining a left table (Integer K: Letter
> > FK,
> > > > > (other
> > > > > > >>>>>>> data))
> > > > > > >>>>>>>> to a right table (Letter K: (some data)).
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Left table:
> > > > > > >>>>>>>> 1: (A, xyz)
> > > > > > >>>>>>>> 2: (B, asd)
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Right table:
> > > > > > >>>>>>>> A: EntityA
> > > > > > >>>>>>>> B: EntityB
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> We could do a rekey as you proposed with a combined key,
> > but
> > > > not
> > > > > > >>>>>>>> propagating the value at all..
> > > > > > >>>>>>>> Rekey table:
> > > > > > >>>>>>>> A-1: (dummy value)
> > > > > > >>>>>>>> B-2: (dummy value)
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Which we then join with the right table to produce:
> > > > > > >>>>>>>> A-1: EntityA
> > > > > > >>>>>>>> B-2: EntityB
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Which gets rekeyed back:
> > > > > > >>>>>>>> 1: A, EntityA
> > > > > > >>>>>>>> 2: B, EntityB
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> And finally we do the actual join:
> > > > > > >>>>>>>> Result table:
> > > > > > >>>>>>>> 1: ((A, xyz), EntityA)
> > > > > > >>>>>>>> 2: ((B, asd), EntityB)
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> The thing is that in that last join, we have the
> > opportunity
> > > > to
> > > > > > >>>>> compare
> > > > > > >>>>>>> the
> > > > > > >>>>>>>> current FK in the left table with the incoming PK of the
> > > right
> > > > > > >>>>> table. If
> > > > > > >>>>>>>> they don't match, we just drop the event, since it must
> be
> > > > > > outdated.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz)
> gets
> > > > > updated
> > > > > > >> to
> > > > > > >>>>>>> (1:
> > > > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether
> the
> > > > final
> > > > > > >>>>> state
> > > > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the
> > algorithm
> > > > > above,
> > > > > > >>>>> you
> > > > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B,
> > > xyz),
> > > > > (B,
> > > > > > >>>>>>>> EntityB)). It seems like this does give you enough
> > > information
> > > > > to
> > > > > > >>>>> make
> > > > > > >>>>>>> the
> > > > > > >>>>>>>> right choice, regardless of disordering.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Will check Adams patch, but this should work. As
> mentioned
> > > > often
> > > > > I
> > > > > > am
> > > > > > >>>>>>> not convinced on partitioning back for the user
> > > automatically.
> > > > I
> > > > > > >> think
> > > > > > >>>>>>> this is the real performance eater ;)
> > > > > > >>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
> > > > performance
> > > > > of
> > > > > > >>>>> the
> > > > > > >>>>>>>> range scans when records change in the right table.
> You've
> > > > said
> > > > > > that
> > > > > > >>>>>>> you've
> > > > > > >>>>>>>> been using the algorithm you presented in production
> for a
> > > > > while.
> > > > > > >> Can
> > > > > > >>>>>>> you
> > > > > > >>>>>>>> give us a sense of the performance characteristics
> you've
> > > > > > observed?
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>> Make it work, make it fast, make it beautiful. The
> topmost
> > > > thing
> > > > > > here
> > > > > > >>>>> is
> > > > > > >>>>>>> / was correctness. In practice I do not measure the
> > > performance
> > > > > of
> > > > > > >> the
> > > > > > >>>>>>> range scan. Usual cases I run this with is emitting 500k
> -
> > > 1kk
> > > > > rows
> > > > > > >>>>>>> on a left hand side change. The range scan is just the
> work
> > > you
> > > > > > gotta
> > > > > > >>>>>>> do, also when you pack your data into different formats,
> > > > usually
> > > > > > the
> > > > > > >>>>>>> rocks performance is very tight to the size of the data
> and
> > > we
> > > > > > can't
> > > > > > >>>>>>> really change that. It is more important for users to
> > prevent
> > > > > > useless
> > > > > > >>>>>>> updates to begin with. My left hand side is guarded to
> drop
> > > > > changes
> > > > > > >>>>> that
> > > > > > >>>>>>> are not going to change my join output.
> > > > > > >>>>>>>
> > > > > > >>>>>>> usually it's:
> > > > > > >>>>>>>
> > > > > > >>>>>>> drop unused fields and then don't forward if
> > old.equals(new)
> > > > > > >>>>>>>
> > > > > > >>>>>>> regarding to the performance of creating an iterator for
> > > > smaller
> > > > > > >>>>>>> fanouts, users can still just do a group by first then
> > > anyways.
> > > > > > >>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>>> I could only think of one alternative, but I'm not sure
> if
> > > > it's
> > > > > > >>>>> better
> > > > > > >>>>>>> or
> > > > > > >>>>>>>> worse... If the first re-key only needs to preserve the
> > > > original
> > > > > > >> key,
> > > > > > >>>>>>> as I
> > > > > > >>>>>>>> proposed in #6, then we could store a vector of keys in
> > the
> > > > > value:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Left table:
> > > > > > >>>>>>>> 1: A,...
> > > > > > >>>>>>>> 2: B,...
> > > > > > >>>>>>>> 3: A,...
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Gets re-keyed:
> > > > > > >>>>>>>> A: [1, 3]
> > > > > > >>>>>>>> B: [2]
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Then, the rhs part of the join would only need a regular
> > > > > > single-key
> > > > > > >>>>>>> lookup.
> > > > > > >>>>>>>> Of course we have to deal with the problem of large
> > values,
> > > as
> > > > > > >>>>> there's
> > > > > > >>>>>>> no
> > > > > > >>>>>>>> bound on the number of lhs records that can reference
> rhs
> > > > > records.
> > > > > > >>>>>>> Offhand,
> > > > > > >>>>>>>> I'd say we could page the values, so when one row is
> past
> > > the
> > > > > > >>>>>>> threshold, we
> > > > > > >>>>>>>> append the key for the next page. Then in most cases, it
> > > would
> > > > > be
> > > > > > a
> > > > > > >>>>>>> single
> > > > > > >>>>>>>> key lookup, but for large fan-out updates, it would be
> one
> > > per
> > > > > > (max
> > > > > > >>>>>>> value
> > > > > > >>>>>>>> size)/(avg lhs key size).
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> This seems more complex, though... Plus, I think there's
> > > some
> > > > > > extra
> > > > > > >>>>>>>> tracking we'd need to do to know when to emit a
> > retraction.
> > > > For
> > > > > > >>>>> example,
> > > > > > >>>>>>>> when record 1 is deleted, the re-key table would just
> have
> > > (A:
> > > > > > [3]).
> > > > > > >>>>>>> Some
> > > > > > >>>>>>>> kind of tombstone is needed so that the join result for
> 1
> > > can
> > > > > also
> > > > > > >> be
> > > > > > >>>>>>>> retracted.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> That's all!
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful
> > KIP.
> > > > > Sorry
> > > > > > >> the
> > > > > > >>>>>>>> discussion has been slow.
> > > > > > >>>>>>>> -John
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > > > > > >>>>> Jan.Filipiak@trivago.com>
> > > > > > >>>>>>>> wrote:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>> Id say you can just call the vote.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> that happens all the time, and if something comes up,
> it
> > > just
> > > > > > goes
> > > > > > >>>>> back
> > > > > > >>>>>>>>> to discuss.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> would not expect to much attention with another another
> > > email
> > > > > in
> > > > > > >>>>> this
> > > > > > >>>>>>>>> thread.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> best Jan
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > > > > > >>>>>>>>>> Hello Contributors
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do need
> > to
> > > > bump
> > > > > > >>>>> this to
> > > > > > >>>>>>>>> keep
> > > > > > >>>>>>>>>> visibility up. I am still intending to push this
> through
> > > > once
> > > > > > >>>>>>> contributor
> > > > > > >>>>>>>>>> feedback is given.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Main points that need addressing:
> > > > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
> > > singular
> > > > > > graph
> > > > > > >>>>> node
> > > > > > >>>>>>>>> into
> > > > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right
> > > now. I
> > > > > am
> > > > > > a
> > > > > > >>>>> bit
> > > > > > >>>>>>>>> fuzzy
> > > > > > >>>>>>>>>> on how the optimizations are supposed to work, so I
> > would
> > > > > > >>>>> appreciate
> > > > > > >>>>>>> any
> > > > > > >>>>>>>>>> help on this aspect.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This
> thread
> > > has
> > > > > > much
> > > > > > >>>>>>>>> discourse
> > > > > > >>>>>>>>>> between Jan and I between the current highwater mark
> > > > proposal
> > > > > > and
> > > > > > >> a
> > > > > > >>>>>>>>> groupBy
> > > > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need to
> > > > > strictly
> > > > > > >>>>> handle
> > > > > > >>>>>>>>> any
> > > > > > >>>>>>>>>> chance of out-of-order data and leave none of it up to
> > the
> > > > > > >>>>> consumer.
> > > > > > >>>>>>> Any
> > > > > > >>>>>>>>>> comments or suggestions here would also help.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> 3) Anything else that you see that would prevent this
> > from
> > > > > > moving
> > > > > > >>>>> to a
> > > > > > >>>>>>>>> vote?
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Thanks
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Adam
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > > > > > >>>>>>>>> adam.bellemare@gmail.com>
> > > > > > >>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>> Hi Jan
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> > > > > > >>>>> Stores.persistentWindowStore,
> > > > > > >>>>>>> you
> > > > > > >>>>>>>>>>> actually only need to specify the amount of segments
> > you
> > > > want
> > > > > > and
> > > > > > >>>>> how
> > > > > > >>>>>>>>> large
> > > > > > >>>>>>>>>>> they are. To the best of my understanding, what
> happens
> > > is
> > > > > that
> > > > > > >>>>> the
> > > > > > >>>>>>>>>>> segments are automatically rolled over as new data
> with
> > > new
> > > > > > >>>>>>> timestamps
> > > > > > >>>>>>>>> are
> > > > > > >>>>>>>>>>> created. We use this exact functionality in some of
> the
> > > > work
> > > > > > done
> > > > > > >>>>>>>>>>> internally at my company. For reference, this is the
> > > > hopping
> > > > > > >>>>> windowed
> > > > > > >>>>>>>>> store.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> In the code that I have provided, there are going to
> be
> > > two
> > > > > 24h
> > > > > > >>>>>>>>> segments.
> > > > > > >>>>>>>>>>> When a record is put into the windowStore, it will be
> > > > > inserted
> > > > > > at
> > > > > > >>>>>>> time
> > > > > > >>>>>>>>> T in
> > > > > > >>>>>>>>>>> both segments. The two segments will always overlap
> by
> > > 12h.
> > > > > As
> > > > > > >>>>> time
> > > > > > >>>>>>>>> goes on
> > > > > > >>>>>>>>>>> and new records are added (say at time T+12h+), the
> > > oldest
> > > > > > >> segment
> > > > > > >>>>>>> will
> > > > > > >>>>>>>>> be
> > > > > > >>>>>>>>>>> automatically deleted and a new segment created. The
> > > > records
> > > > > > are
> > > > > > >>>>> by
> > > > > > >>>>>>>>> default
> > > > > > >>>>>>>>>>> inserted with the context.timestamp(), such that it
> is
> > > the
> > > > > > record
> > > > > > >>>>>>> time,
> > > > > > >>>>>>>>> not
> > > > > > >>>>>>>>>>> the clock time, which is used.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> To the best of my understanding, the timestamps are
> > > > retained
> > > > > > when
> > > > > > >>>>>>>>>>> restoring from the changelog.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL
> > at a
> > > > > > >>>>>>> segment-level,
> > > > > > >>>>>>>>>>> instead of at an individual record level.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > > > > > >>>>>>> Jan.Filipiak@trivago.com>
> > > > > > >>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
> > > > > > ClassCastException
> > > > > > >>>>> or
> > > > > > >>>>>>>>>>>> similar.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> You either would have to specify the window you
> > > fetch/put
> > > > or
> > > > > > >>>>> iterate
> > > > > > >>>>>>>>>>>> across all windows the key was found in right?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> I just hope the window-store doesn't check
> stream-time
> > > > under
> > > > > > the
> > > > > > >>>>>>> hoods
> > > > > > >>>>>>>>>>>> that would be a questionable interface.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> If it does: did you see my comment on checking all
> the
> > > > > windows
> > > > > > >>>>>>> earlier?
> > > > > > >>>>>>>>>>>> that would be needed to actually give reasonable
> time
> > > > > > gurantees.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Best
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > > > > > >>>>>>>>>>>>> Hi Jan
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only
> changed
> > > the
> > > > > > state
> > > > > > >>>>>>> store,
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>> the ProcessorSupplier.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>> Adam
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> @Guozhang
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed
> > something
> > > > that
> > > > > > >>>>> will be
> > > > > > >>>>>>>>>>>>>>> extremely
> > > > > > >>>>>>>>>>>>>>> useful for this KIP.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> @Jan
> > > > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I
> > will
> > > > not
> > > > > > be
> > > > > > >>>>>>> moving
> > > > > > >>>>>>>>>>>> ahead
> > > > > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy
> > > solution
> > > > > as
> > > > > > >> you
> > > > > > >>>>>>>>>>>> propose.
> > > > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it
> > yourself
> > > > off
> > > > > > of
> > > > > > >>>>> my
> > > > > > >>>>>>>>>>>> current PR
> > > > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I
> would
> > > be
> > > > > more
> > > > > > >>>>> than
> > > > > > >>>>>>>>>>>> happy to
> > > > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> > stands
> > > > > right
> > > > > > >>>>> now,
> > > > > > >>>>>>> I do
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>> really have more time to invest into alternatives
> > > > without
> > > > > > >>>>> there
> > > > > > >>>>>>>>> being
> > > > > > >>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>> strong indication from the binding voters which
> > they
> > > > > would
> > > > > > >>>>>>> prefer.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave
> up
> > on
> > > > the
> > > > > > >>>>> streams
> > > > > > >>>>>>>>> DSL
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>> some time already, otherwise I would have pulled
> > this
> > > > KIP
> > > > > > >>>>> through
> > > > > > >>>>>>>>>>>> already.
> > > > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on
> > > PAPI.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the
> windowed
> > > > state
> > > > > > >>>>> store
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I
> > will
> > > > come
> > > > > > >> back
> > > > > > >>>>>>> for
> > > > > > >>>>>>>>>>>> final
> > > > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of
> > the
> > > > > > binding
> > > > > > >>>>>>> voters
> > > > > > >>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have
> updated
> > it
> > > > > > >>>>> according
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> latest plan:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > > > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed
> > > store.
> > > > > > This
> > > > > > >>>>>>> could
> > > > > > >>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they
> > are
> > > > > > >>>>> completed.
> > > > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Adam
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> > already
> > > > > > updated
> > > > > > >>>>> in
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> PR?
> > > > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
> > > > > something?
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > > > > > >>>>>>> wangguoz@gmail.com>
> > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is
> the
> > > > wrong
> > > > > > >> link,
> > > > > > >>>>>>> as it
> > > > > > >>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part
> of
> > > > > KIP-258
> > > > > > >>>>> we do
> > > > > > >>>>>>>>>>>> want to
> > > > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source
> > KTable"
> > > > such
> > > > > > >> that
> > > > > > >>>>>>>>>>>> instead of
> > > > > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized
> > store,
> > > > > i.e.
> > > > > > >>>>>>> following
> > > > > > >>>>>>>>>>>>>>>> offset
> > > > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older
> > than
> > > > the
> > > > > > >>>>> current
> > > > > > >>>>>>>>>>>> key's
> > > > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Guozhang
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang
> <
> > > > > > >>>>>>>>> wangguoz@gmail.com>
> > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Hello Adam,
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final
> > > step
> > > > > > (i.e.
> > > > > > >>>>> the
> > > > > > >>>>>>>>> high
> > > > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced
> with
> > a
> > > > > window
> > > > > > >>>>>>> store),
> > > > > > >>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>> think
> > > > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > >>>>>>>>>>>>>>>>>
> > 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a
> key-value
> > > > store
> > > > > > >>>>> (i.e.
> > > > > > >>>>>>> only
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its
> usage,
> > as
> > > > > > >>>>> described
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>
> https://issues.apache.org/jira/browse/KAFKA-5533
> > ,
> > > is
> > > > > > that
> > > > > > >>>>> we
> > > > > > >>>>>>> can
> > > > > > >>>>>>>>>>>> then
> > > > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
> > > > > timestamp
> > > > > > is
> > > > > > >>>>>>>>> smaller
> > > > > > >>>>>>>>>>>> than
> > > > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I
> > think
> > > it
> > > > > is
> > > > > > >>>>> very
> > > > > > >>>>>>>>>>>> similar to
> > > > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
> > > > > filtering,
> > > > > > >>>>> while
> > > > > > >>>>>>>>> you
> > > > > > >>>>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the
> > > joining
> > > > > > >> records
> > > > > > >>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> correctly
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the
> final
> > > > stage.
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and
> > hence
> > > > > > >>>>>>> non-windowed
> > > > > > >>>>>>>>>>>> KTables
> > > > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really
> > > have
> > > > a
> > > > > > good
> > > > > > >>>>>>>>> support
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>> their joins anyways (
> > > > > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > > > > > >>>>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
> > > KTable-KTable
> > > > > > >>>>> non-key
> > > > > > >>>>>>>>> joins
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> Guozhang
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > > > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would
> > grow
> > > > > > >> endlessly
> > > > > > >>>>>>> based
> > > > > > >>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair
> of
> > > > (<this
> > > > > > >>>>> table
> > > > > > >>>>>>>>>>>> primary
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> key>,
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is
> used
> > > to
> > > > > > >>>>>>> differentiate
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> between
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest
> proposal
> > > > would
> > > > > > be
> > > > > > >>>>> to
> > > > > > >>>>>>>>>>>> replace
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This
> > > would
> > > > > > allow
> > > > > > >>>>> the
> > > > > > >>>>>>>>> same
> > > > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time.
> This
> > > > > should
> > > > > > >>>>> allow
> > > > > > >>>>>>> for
> > > > > > >>>>>>>>>>>> all
> > > > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> > should
> > > be
> > > > > > >>>>>>> customizable
> > > > > > >>>>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie:
> perhaps
> > > just
> > > > > 10
> > > > > > >>>>>>> minutes
> > > > > > >>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> window,
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do
> the
> > > > trick
> > > > > > >> here.
> > > > > > >>>>>>> Even
> > > > > > >>>>>>>>>>>> if I
> > > > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
> > > repartitioning
> > > > > > >>>>> optional
> > > > > > >>>>>>>>>>>> since I
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am
> a
> > > > little
> > > > > > bit
> > > > > > >>>>>>>>>>>> sceptical
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one
> > > could
> > > > > run
> > > > > > >>>>> into
> > > > > > >>>>>>>>>>>> problems
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window border.
> I
> > > will
> > > > > > check
> > > > > > >>>>> you
> > > > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic,
> we
> > > > could
> > > > > > >>>>> still
> > > > > > >>>>>>>>> check
> > > > > > >>>>>>>>>>>>>>>>>> _all_
> > > > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance
> > > impact I
> > > > > > >> guess.
> > > > > > >>>>>>> Will
> > > > > > >>>>>>>>>>>> let
> > > > > > >>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct as
> > > is. I
> > > > > > >>>>> wouldn't
> > > > > > >>>>>>> not
> > > > > > >>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> > > timestamp(A)  <
> > > > > > >>>>>>>>> timestamp(B).
> > > > > > >>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> think
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> we can't expect that.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> @Jan
> > > > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now -
> > thanks
> > > > for
> > > > > > the
> > > > > > >>>>>>>>>>>> diagram, it
> > > > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do
> not
> > > have
> > > > > the
> > > > > > >>>>>>> original
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> primary
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
> > > available
> > > > > > then
> > > > > > >>>>> you
> > > > > > >>>>>>>>>>>> would be
> > > > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map.
> > That
> > > > > being
> > > > > > >>>>> said,
> > > > > > >>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> encourage
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for
> > > clarity
> > > > > for
> > > > > > >>>>>>> everyone
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> else.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really
> hard
> > > > work.
> > > > > > But
> > > > > > >>>>> I
> > > > > > >>>>>>>>>>>> understand
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> > > original
> > > > > > >> primary
> > > > > > >>>>>>> key,
> > > > > > >>>>>>>>> We
> > > > > > >>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI
> > and
> > > > > > >> basically
> > > > > > >>>>>>> not
> > > > > > >>>>>>>>>>>> using
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed
> > that
> > > in
> > > > > > >>>>> original
> > > > > > >>>>>>> DSL
> > > > > > >>>>>>>>>>>> its
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up
> on
> > > my
> > > > > end.
> > > > > > >>>>> Will
> > > > > > >>>>>>>>>>>> finish
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this
> week.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the
> Map
> > > stay
> > > > > > >> inside
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> State
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes
> > have
> > > > > > >>>>> propagated?
> > > > > > >>>>>>>>> Isn't
> > > > > > >>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark
> state
> > > > store?
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty,
> substractor
> > is
> > > > > gonna
> > > > > > >>>>>>> return
> > > > > > >>>>>>>>>>>> `null`
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there
> > is
> > > > > going
> > > > > > to
> > > > > > >>>>> be
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this
> > store
> > > > > > directly
> > > > > > >>>>> for
> > > > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a
> > > > regular
> > > > > > >>>>> store,
> > > > > > >>>>>>>>>>>> satisfying
> > > > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby /
> join.
> > > The
> > > > > > >>>>> Windowed
> > > > > > >>>>>>>>>>>> store is
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull
> > > > operation
> > > > > > we
> > > > > > >>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have
> > the
> > > > > > window
> > > > > > >>>>>>> store
> > > > > > >>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> the values then.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom
> > group
> > > > by
> > > > > > >>>>> before
> > > > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i
> > think
> > > > it
> > > > > > >> would
> > > > > > >>>>>>> help
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> users
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the
> > > > original
> > > > > > >>>>> primary
> > > > > > >>>>>>>>> key
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> issue I
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid
> foundation
> > > to
> > > > > > build
> > > > > > >>>>> on.
> > > > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user.
> > very
> > > > > > >>>>>>> unfortunate. I
> > > > > > >>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do
> not
> > > > think
> > > > > > its
> > > > > > >>>>> a
> > > > > > >>>>>>> good
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> decision.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Thanks
> > > > > > >>>>>>>>>>>>>>>>>>> Adam
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta
> > > Dumbre <
> > > > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > > > > > >>>>>>> dumbreprajakta311@gmail.com
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan
> > > > > Filipiak
> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> > > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make
> such a
> > > > > chart.
> > > > > > >> last
> > > > > > >>>>>>> time i
> > > > > > >>>>>>>>>>>> didn't
> > > > > > >>>>>>>>>>>>>>>>>>>           get along
> > > > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all
> your
> > > > > charts.
> > > > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> > > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam
> Bellemare
> > > > > wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of
> > what I
> > > > > > >> attempted
> > > > > > >>>>> on
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>
> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > > > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > > > > > >>>>>>>>>>>>>>>>>>>           <
> > > > > > >>>>>>>>>>>>
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>
> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > > > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the
> > start
> > > of
> > > > > my
> > > > > > own
> > > > > > >>>>>>>>>>>> implementation
> > > > > > >>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>           this
> > > > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not
> > get
> > > > it
> > > > > to
> > > > > > >>>>> work I
> > > > > > >>>>>>> have
> > > > > > >>>>>>>>>>>> since
> > > > > > >>>>>>>>>>>>>>>>>>>           discarded the
> > > > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if
> > you
> > > > > wish
> > > > > > to
> > > > > > >>>>>>> continue
> > > > > > >>>>>>>>>>>> pursuing
> > > > > > >>>>>>>>>>>>>>>>>>>           for your
> > > > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that
> you
> > > > please
> > > > > > >>>>> create a
> > > > > > >>>>>>>>>>>> diagram on
> > > > > > >>>>>>>>>>>>>>>>>>>           the KIP
> > > > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your
> > solution.
> > > > > > Please
> > > > > > >>>>> feel
> > > > > > >>>>>>> free
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> use
> > > > > > >>>>>>>>>>>>>>>>>>>           the image I
> > > > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting
> point.
> > I
> > > am
> > > > > > having
> > > > > > >>>>>>> trouble
> > > > > > >>>>>>>>>>>>>>>>>>>           understanding your
> > > > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a
> > > > > carefully
> > > > > > >>>>>>> constructed
> > > > > > >>>>>>>>>>>> diagram
> > > > > > >>>>>>>>>>>>>>>>>>>           will clear
> > > > > > >>>>>>>>>>>>>>>>>>>           > up
> > > > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings.
> > Alternately,
> > > > > > please
> > > > > > >>>>> post a
> > > > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> > > > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess
> > at
> > > > what
> > > > > > you
> > > > > > >>>>>>> mean, and
> > > > > > >>>>>>>>>>>> since I
> > > > > > >>>>>>>>>>>>>>>>>>>           value my
> > > > > > >>>>>>>>>>>>>>>>>>>           > own
> > > > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value
> yours,
> > I
> > > > > > believe
> > > > > > >> it
> > > > > > >>>>> is
> > > > > > >>>>>>> your
> > > > > > >>>>>>>>>>>>>>>>>>>           responsibility to
> > > > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation
> instead
> > > of
> > > > me
> > > > > > >>>>> trying to
> > > > > > >>>>>>>>> guess.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > > Adam
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM,
> > Jan
> > > > > > Filipiak
> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing
> > interested.
> > > > > kafka
> > > > > > >>>>>>> streams at
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>           point supports
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as
> > both
> > > > > > streams
> > > > > > >>>>> have
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> same
> > > > > > >>>>>>>>>>>>>>>>>>> key.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently
> implementing a
> > > > join
> > > > > > >> where a
> > > > > > >>>>>>> KTable
> > > > > > >>>>>>>>>>>> and a
> > > > > > >>>>>>>>>>>>>>>>>>>           KTable can
> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship
> > (1:n).
> > > > We
> > > > > > >> exploit
> > > > > > >>>>>>> that
> > > > > > >>>>>>>>>>>> rocksdb
> > > > > > >>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted
> > (At
> > > > > least
> > > > > > >>>>>>> exposes an
> > > > > > >>>>>>>>>>>> API to
> > > > > > >>>>>>>>>>>>>>>>>>>           access the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted
> > fashion).
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats
> > are
> > > > well
> > > > > > >>>>>>> understood
> > > > > > >>>>>>>>> now
> > > > > > >>>>>>>>>>>> and we
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > basically
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API
> > Design
> > > (
> > > > > when
> > > > > > >> Adam
> > > > > > >>>>>>> sees
> > > > > > >>>>>>>>> my
> > > > > > >>>>>>>>>>>> newest
> > > > > > >>>>>>>>>>>>>>>>>>>           message).
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record
> of
> > > > > loosing
> > > > > > >>>>> those
> > > > > > >>>>>>> kinda
> > > > > > >>>>>>>>>>>>>>>>>>>           arguments within
> > > > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have
> no
> > > clue
> > > > > > why.
> > > > > > >> So
> > > > > > >>>>> I
> > > > > > >>>>>>>>>>>> literally
> > > > > > >>>>>>>>>>>>>>>>>>>           can't wait for
> > > > > > >>>>>>>>>>>>>>>>>>>           > you
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread
> and
> > > > give
> > > > > > you
> > > > > > >>>>>>> opinion on
> > > > > > >>>>>>>>>>>> how we
> > > > > > >>>>>>>>>>>>>>>>>>>           should
> > > > > > >>>>>>>>>>>>>>>>>>>           > design
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
> > > oneToManyJoin
> > > > > and
> > > > > > >> how
> > > > > > >>>>>>> many
> > > > > > >>>>>>>>>>>> power we
> > > > > > >>>>>>>>>>>>>>>>>>>           want to give
> > > > > > >>>>>>>>>>>>>>>>>>>           > to
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity"
> (where
> > > > > > simplicity
> > > > > > >>>>> isn't
> > > > > > >>>>>>>>>>>> really that
> > > > > > >>>>>>>>>>>>>>>>>>>           as users
> > > > > > >>>>>>>>>>>>>>>>>>>           > still
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on
> > the
> > > > > > >> discussion
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James
> Kwan
> > > > wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I
> > > found
> > > > > this
> > > > > > >>>>> subject
> > > > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> > > > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a
> > join
> > > > > > table of
> > > > > > >>>>> two
> > > > > > >>>>>>>>>>>> streams? Is
> > > > > > >>>>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>           > somewhere
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
> > > requirement
> > > > or
> > > > > > >>>>> proposal?
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM,
> Jan
> > > > > > Filipiak
> > > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
> > > > Bellemare
> > > > > > >> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing
> using a
> > > > > > Windowed
> > > > > > >>>>> Store
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>>>> store the
> > > > > > >>>>>>>>>>>>>>>>>>>           highwater
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this
> > should
> > > > work
> > > > > > >> fine,
> > > > > > >>>>>>> with
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> caveat
> > > > > > >>>>>>>>>>>>>>>>>>>           being that
> > > > > > >>>>>>>>>>>>>>>>>>>           > it
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order
> > > arrival
> > > > > > for up
> > > > > > >>>>> to
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> size of
> > > > > > >>>>>>>>>>>>>>>>>>>           the window
> > > > > > >>>>>>>>>>>>>>>>>>>           > (ie:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would
> > > remove
> > > > > the
> > > > > > >>>>>>> possibility
> > > > > > >>>>>>>>>>>> of it
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> being
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > unbounded
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
> > > > suggestion, I
> > > > > > >>>>> believe
> > > > > > >>>>>>> this
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>>> where
> > > > > > >>>>>>>>>>>>>>>>>>>           we will
> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement.
> > While I
> > > > do
> > > > > > not
> > > > > > >>>>>>> disagree
> > > > > > >>>>>>>>>>>> with your
> > > > > > >>>>>>>>>>>>>>>>>>>           statement
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be
> additional
> > > > joins
> > > > > > done
> > > > > > >>>>> in a
> > > > > > >>>>>>>>>>>> real-world
> > > > > > >>>>>>>>>>>>>>>>>>>           workflow, I
> > > > > > >>>>>>>>>>>>>>>>>>>           > do
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can
> conclusively
> > > deal
> > > > > > with
> > > > > > >>>>>>>>> out-of-order
> > > > > > >>>>>>>>>>>>>>>>>>> arrival
> > > > > > >>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent
> > joins. I
> > > > > have
> > > > > > >>>>>>> attempted
> > > > > > >>>>>>>>> what
> > > > > > >>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>           think you have
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a
> > high-water,
> > > > > using
> > > > > > >>>>>>> groupBy and
> > > > > > >>>>>>>>>>>> reduce)
> > > > > > >>>>>>>>>>>>>>>>>>>           and found
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too
> > > > > quickly,
> > > > > > or
> > > > > > >>>>> the
> > > > > > >>>>>>> load
> > > > > > >>>>>>>>> on
> > > > > > >>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>           stream thread
> > > > > > >>>>>>>>>>>>>>>>>>>           > is
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages
> > will
> > > > > arrive
> > > > > > >>>>>>>>> out-of-order
> > > > > > >>>>>>>>>>>> and be
> > > > > > >>>>>>>>>>>>>>>>>>>           incorrectly
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> > > > > intermediate
> > > > > > >>>>> event
> > > > > > >>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>> represented
> > > > > > >>>>>>>>>>>>>>>>>>>           as the
> > > > > > >>>>>>>>>>>>>>>>>>>           > final
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on
> > your
> > > > > > groupBy
> > > > > > >>>>>>>>>>>> implementation.
> > > > > > >>>>>>>>>>>>>>>>>>>           There must be
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it
> > is,
> > > I
> > > > > > would
> > > > > > >>>>> just
> > > > > > >>>>>>> like
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must
> be
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I
> > > would
> > > > > > like
> > > > > > >> to
> > > > > > >>>>>>> clarify
> > > > > > >>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>           we draw a
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the
> > scattered
> > > > > events
> > > > > > >>>>> back to
> > > > > > >>>>>>>>> their
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> original
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I
> > know
> > > > how
> > > > > > to
> > > > > > >>>>>>>>> conclusively
> > > > > > >>>>>>>>>>>> deal
> > > > > > >>>>>>>>>>>>>>>>>>>           with
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a
> > given
> > > > time
> > > > > > >> frame,
> > > > > > >>>>>>> and to
> > > > > > >>>>>>>>>>>> ensure
> > > > > > >>>>>>>>>>>>>>>>>>>           that the
> > > > > > >>>>>>>>>>>>>>>>>>>           > data
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with
> > the
> > > > > input
> > > > > > >>>>> events.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to
> > share
> > > > that
> > > > > > >>>>>>> illustrates
> > > > > > >>>>>>>>> your
> > > > > > >>>>>>>>>>>>>>>>>>>           approach, I
> > > > > > >>>>>>>>>>>>>>>>>>>           > would
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would
> > > remove
> > > > > any
> > > > > > >>>>>>>>>>>> misunderstandings
> > > > > > >>>>>>>>>>>>>>>>>>>           that I may
> > > > > > >>>>>>>>>>>>>>>>>>>           > have.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for
> > my
> > > > > code.
> > > > > > I
> > > > > > >>>>> don't
> > > > > > >>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>           something easily
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated
> > > with
> > > > > > >>>>> OO-patterns.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K
> aggKey,
> > V
> > > > > > value, T
> > > > > > >>>>>>>>> aggregate)
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> > > > > currentStateAsMap =
> > > > > > >>>>>>>>>>>> asMap(aggregate);
> > > > > > >>>>>>>>>>>>>>>>>>> <<
> > > > > > >>>>>>>>>>>>>>>>>>>           imaginary
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> > > > > > >>>>> mapper.apply(value);
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the
> > > place
> > > > > > where
> > > > > > >>>>> people
> > > > > > >>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>>>>>>>>           gonna have
> > > > > > >>>>>>>>>>>>>>>>>>>           > issues
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably
> couldn't
> > do
> > > > it.
> > > > > > we
> > > > > > >>>>> would
> > > > > > >>>>>>> need
> > > > > > >>>>>>>>>>>> to find
> > > > > > >>>>>>>>>>>>>>>>>>>           a solution
> > > > > > >>>>>>>>>>>>>>>>>>>           > here.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate
> > the
> > > > > > field in
> > > > > > >>>>> the
> > > > > > >>>>>>>>>>>> joiner, so
> > > > > > >>>>>>>>>>>>>>>>>>>           that we can
> > > > > > >>>>>>>>>>>>>>>>>>>           > pick
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate.
> > Probably
> > > > you
> > > > > > have
> > > > > > >>>>> not
> > > > > > >>>>>>>>> thought
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>           this in your
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very
> open
> > > to
> > > > > > find a
> > > > > > >>>>>>> generic
> > > > > > >>>>>>>>>>>> solution
> > > > > > >>>>>>>>>>>>>>>>>>>           here. In my
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is
> broken
> > in
> > > > > > >>>>>>>>> KTableImpl.GroupBy
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>           looses
> > > > > > >>>>>>>>>>>>>>>>>>>           > the keys
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the
> > aggregate
> > > > > key.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted
> > it
> > > > away
> > > > > > back
> > > > > > >>>>>>> then way
> > > > > > >>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>> i
> > > > > > >>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>           > thinking
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is
> > why I
> > > > > > didn't
> > > > > > >>>>>>> realize
> > > > > > >>>>>>>>> its
> > > > > > >>>>>>>>>>>>>>>>>>>           significance here.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > currentStateAsMap.put(mapper.apply(m),
> > > > > > >> m);
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > currentStateAsMap.put(toModifyKey,
> > > > > > >> value);
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > currentStateAsMap.remove(toModifyKey);
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > if(currentStateAsMap.isEmpty()){
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return
> null;
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> > > > > > >>>>>>> asAggregateType(currentStateAsMap)
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35
> > PM,
> > > > Jan
> > > > > > >>>>> Filipiak
> > > > > > >>>>>>> <
> > > > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com
> <mailto:
> > > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing
> > > Matthias
> > > > > to
> > > > > > >>>>> speed!
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I
> > think
> > > > > > >> re-keying
> > > > > > >>>>>>> back
> > > > > > >>>>>>>>>>>> should be
> > > > > > >>>>>>>>>>>>>>>>>>>           optional at
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> > > > > > KScatteredTable
> > > > > > >>>>> with
> > > > > > >>>>>>>>>>>> reshuffle()
> > > > > > >>>>>>>>>>>>>>>>>>>           returning
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined>
> > to
> > > > make
> > > > > > the
> > > > > > >>>>>>> backwards
> > > > > > >>>>>>>>>>>>>>>>>>>           repartitioning
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour
> of
> > > > doing
> > > > > > the
> > > > > > >>>>> out
> > > > > > >>>>>>> of
> > > > > > >>>>>>>>> order
> > > > > > >>>>>>>>>>>>>>>>>>>           processing using
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark
> > > > > tracking.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded
> > growth
> > > is
> > > > > > just
> > > > > > >>>>> scary
> > > > > > >>>>>>> + It
> > > > > > >>>>>>>>>>>> saves
> > > > > > >>>>>>>>>>>>>>>>>>> us
> > > > > > >>>>>>>>>>>>>>>>>>>           the header
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of
> > > > always
> > > > > > >>>>>>> repartitioning
> > > > > > >>>>>>>>>>>> back is
> > > > > > >>>>>>>>>>>>>>>>>>>           just not so
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has
> > been
> > > > > done
> > > > > > >>>>> before
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>           back and
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else
> afterwards
> > > is
> > > > > > really
> > > > > > >>>>>>> common.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
> > > > > Bellemare
> > > > > > >>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your
> > feedback,
> > > I
> > > > do
> > > > > > >>>>>>> appreciate
> > > > > > >>>>>>>>> it!
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would
> be
> > > > > > possible,
> > > > > > >> it
> > > > > > >>>>>>> would
> > > > > > >>>>>>>>>>>> require
> > > > > > >>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           > deserialize
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
> implies
> > a
> > > > > > runtime
> > > > > > >>>>>>> overhead.
> > > > > > >>>>>>>>> I
> > > > > > >>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
> avoid
> > > the
> > > > > > >>>>> overhead.
> > > > > > >>>>>>> If
> > > > > > >>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still
> > add
> > > > > name
> > > > > > >>>>> spacing
> > > > > > >>>>>>>>> later
> > > > > > >>>>>>>>>>>> on.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with
> > > using a
> > > > > > >> reserved
> > > > > > >>>>>>> string
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>           document it.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the
> > > > design
> > > > > it
> > > > > > >> the
> > > > > > >>>>>>> type of
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
> > > > > correctly,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have
> > > > table1
> > > > > > and
> > > > > > >>>>> table2
> > > > > > >>>>>>>>>>>> swapped.
> > > > > > >>>>>>>>>>>>>>>>>>>           Here is how it
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records
> > > that
> > > > > > contain
> > > > > > >>>>> the
> > > > > > >>>>>>>>>>>> foreign key
> > > > > > >>>>>>>>>>>>>>>>>>>           within their
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> > > > > > <a,(fk=A,bar=1)>,
> > > > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream:
> <A,X>,
> > > > <B,Y>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is
> > required
> > > > to
> > > > > > >> extract
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> foreign
> > > > > > >>>>>>>>>>>>>>>>>>> key.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key
> mapper:
> > (
> > > > > value
> > > > > > =>
> > > > > > >>>>>>> value.fk
> > > > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to
> > each
> > > > > > element
> > > > > > >> in
> > > > > > >>>>>>>>> table1,
> > > > > > >>>>>>>>>>>> and a
> > > > > > >>>>>>>>>>>>>>>>>>>           new combined
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> > > > > (fk=A,bar=1)>,
> > > > > > >>>>> <A-b,
> > > > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > > > > > >>>>>>>>>>>>>>>>>>>           <B-c,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> > > > > > >> copartitioned
> > > > > > >>>>>>> with
> > > > > > >>>>>>>>>>>> table2:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
> > > Partition
> > > > > 0:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
> <A-a,
> > > > > > >>>>> (fk=A,bar=1)>,
> > > > > > >>>>>>> <A-b,
> > > > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
> > > Partition
> > > > > 1:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1:
> <B-c,
> > > > > > >> (fk=B,bar=3)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be
> > > > joined
> > > > > > >>>>> together
> > > > > > >>>>>>>>> locally
> > > > > > >>>>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>>>>>>>           applying the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's
> design
> > > and
> > > > > my
> > > > > > >>>>> design
> > > > > > >>>>>>>>>>>> deviate. My
> > > > > > >>>>>>>>>>>>>>>>>>>           design goes
> > > > > > >>>>>>>>>>>>>>>>>>>           > on
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
> > > post-join
> > > > > and
> > > > > > >>>>> resolve
> > > > > > >>>>>>>>>>>> out-of-order
> > > > > > >>>>>>>>>>>>>>>>>>>           arrival of
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the
> data
> > > > keyed
> > > > > > just
> > > > > > >>>>> the
> > > > > > >>>>>>>>>>>> original key.
> > > > > > >>>>>>>>>>>>>>>>>>>           I do not
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the
> > > > > internals
> > > > > > >>>>>>> outside of
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make
> > for
> > > > > larger
> > > > > > >>>>>>> footprint,
> > > > > > >>>>>>>>>>>> but it
> > > > > > >>>>>>>>>>>>>>>>>>>           removes all
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving
> out-of-order
> > > > > arrivals
> > > > > > >> and
> > > > > > >>>>>>>>> handling
> > > > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> > > > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this
> > > makes
> > > > > the
> > > > > > >>>>>>> function
> > > > > > >>>>>>>>> much
> > > > > > >>>>>>>>>>>>>>>>>>> easier
> > > > > > >>>>>>>>>>>>>>>>>>>           to use.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps
> > > > resolve
> > > > > > your
> > > > > > >>>>>>>>> questions,
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>           please feel
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your
> > > mind.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at
> 8:36
> > > PM,
> > > > > > >>>>> Matthias J.
> > > > > > >>>>>>>>> Sax <
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io
> > > <mailto:
> > > > > > >>>>>>>>>>>> matthias@confluent.io>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on
> > > this
> > > > > > >> thread. I
> > > > > > >>>>>>> did
> > > > > > >>>>>>>>> not
> > > > > > >>>>>>>>>>>> read
> > > > > > >>>>>>>>>>>>>>>>>>>           everything so
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple
> > of
> > > > > > initial
> > > > > > >>>>>>> thoughts:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there
> is
> > a
> > > > > > >> fundamental
> > > > > > >>>>>>>>>>>> difference
> > > > > > >>>>>>>>>>>>>>>>>>>           between header
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258.
> > For
> > > > 258,
> > > > > > we
> > > > > > >> add
> > > > > > >>>>>>>>> headers
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka
> Streams
> > > and
> > > > > > nobody
> > > > > > >>>>>>> else is
> > > > > > >>>>>>>>>>>> supposed
> > > > > > >>>>>>>>>>>>>>>>>>>           to write
> > > > > > >>>>>>>>>>>>>>>>>>>           > into
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user
> > > header
> > > > > are
> > > > > > >>>>> written
> > > > > > >>>>>>> into
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
> > > conflicts.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't
> see
> > a
> > > > big
> > > > > > issue
> > > > > > >>>>> with
> > > > > > >>>>>>>>> using
> > > > > > >>>>>>>>>>>>>>>>>>>           headers within
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document
> it,
> > > we
> > > > > can
> > > > > > >> have
> > > > > > >>>>>>> some
> > > > > > >>>>>>>>>>>> "reserved"
> > > > > > >>>>>>>>>>>>>>>>>>>           header keys
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to
> > use
> > > > > when
> > > > > > >>>>>>> processing
> > > > > > >>>>>>>>>>>> data with
> > > > > > >>>>>>>>>>>>>>>>>>>           Kafka
> > > > > > >>>>>>>>>>>>>>>>>>>           > Streams.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe
> > way
> > > to
> > > > > > avoid
> > > > > > >>>>>>>>> conflicts,
> > > > > > >>>>>>>>>>>> since
> > > > > > >>>>>>>>>>>>>>>>>>> these
> > > > > > >>>>>>>>>>>>>>>>>>>           > headers
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal
> > > > topics
> > > > > (I
> > > > > > >>>>> think):
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and
> > changelog
> > > > > > topics,
> > > > > > >> we
> > > > > > >>>>> can
> > > > > > >>>>>>>>>>>> namespace
> > > > > > >>>>>>>>>>>>>>>>>>>           all headers:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers
> > are
> > > > > > >> namespaced
> > > > > > >>>>> as
> > > > > > >>>>>>>>>>>> "external."
> > > > > > >>>>>>>>>>>>>>>>>>> +
> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> > > > > > namespaced as
> > > > > > >>>>>>>>>>>> "internal." +
> > > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing
> would
> > be
> > > > > > >> possible,
> > > > > > >>>>> it
> > > > > > >>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>> require
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what
> implies
> > a
> > > > > > runtime
> > > > > > >>>>>>> overhead.
> > > > > > >>>>>>>>> I
> > > > > > >>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to
> avoid
> > > the
> > > > > > >>>>> overhead.
> > > > > > >>>>>>> If
> > > > > > >>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still
> > add
> > > > > name
> > > > > > >>>>> spacing
> > > > > > >>>>>>>>> later
> > > > > > >>>>>>>>>>>> on.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about
> the
> > > > design
> > > > > > it
> > > > > > >> the
> > > > > > >>>>>>> type
> > > > > > >>>>>>>>> of
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
> > > > > correctly,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 =
> ...
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 =
> ...
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3>
> joinedTable
> > =
> > > > > > >>>>>>>>>>>> table1.join(table2,...);
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
> > > `joinedTable`
> > > > > has
> > > > > > >> the
> > > > > > >>>>>>> same
> > > > > > >>>>>>>>> key
> > > > > > >>>>>>>>>>>> as the
> > > > > > >>>>>>>>>>>>>>>>>>>           left input
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work
> > > > because
> > > > > > if
> > > > > > >>>>> table2
> > > > > > >>>>>>>>>>>> contains
> > > > > > >>>>>>>>>>>>>>>>>>>           multiple rows
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in
> > table1
> > > > > > (what is
> > > > > > >>>>> the
> > > > > > >>>>>>> main
> > > > > > >>>>>>>>>>>> purpose
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>           > foreign
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table
> > > would
> > > > > only
> > > > > > >>>>>>> contain a
> > > > > > >>>>>>>>>>>> single
> > > > > > >>>>>>>>>>>>>>>>>>>           join result,
> > > > > > >>>>>>>>>>>>>>>>>>>           > but
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream:
> <A,X>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
> > > > <a,(A,1)>,
> > > > > > >>>>> <b,(A,2)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a
> > > foreign
> > > > > key
> > > > > > to
> > > > > > >>>>>>> table1
> > > > > > >>>>>>>>> key
> > > > > > >>>>>>>>>>>> (ie,
> > > > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
> > > > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same
> key
> > > as
> > > > > key
> > > > > > of
> > > > > > >>>>>>> table1,
> > > > > > >>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>           implies that the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> > > > > > join(X,1)>
> > > > > > >> or
> > > > > > >>>>> <A,
> > > > > > >>>>>>>>>>>> join(X,2)>
> > > > > > >>>>>>>>>>>>>>>>>>>           but not
> > > > > > >>>>>>>>>>>>>>>>>>>           > both.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the
> same
> > > > key,
> > > > > > >>>>> whatever
> > > > > > >>>>>>>>> result
> > > > > > >>>>>>>>>>>> record
> > > > > > >>>>>>>>>>>>>>>>>>>           we emit
> > > > > > >>>>>>>>>>>>>>>>>>>           > later,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous
> > > result.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why
> Jan
> > > > > > originally
> > > > > > >>>>>>> proposed
> > > > > > >>>>>>>>>>>> to use
> > > > > > >>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>           > combination
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the
> > > input
> > > > > > tables
> > > > > > >>>>> as
> > > > > > >>>>>>> key
> > > > > > >>>>>>>>> of
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           output table.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the
> > output
> > > > > table
> > > > > > >>>>> unique
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>> we
> > > > > > >>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>           store both in
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> > > > > join(X,1)>,
> > > > > > >> <A-b,
> > > > > > >>>>>>>>>>>> join(X,2)>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan
> > > > Filipiak
> > > > > > >> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark
> could
> > be
> > > > > > >>>>> disregarded.
> > > > > > >>>>>>> The
> > > > > > >>>>>>>>>>>> decision
> > > > > > >>>>>>>>>>>>>>>>>>>           about the
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of
> > the
> > > > > > >> aggregated
> > > > > > >>>>>>> map.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long
> maps
> > > > would
> > > > > be
> > > > > > >>>>>>> unpacked
> > > > > > >>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> > > > > > >>>>>>>>>>>>>>>>>>>           > element
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as
> > > delete.
> > > > > Any
> > > > > > >>>>> other
> > > > > > >>>>>>> count
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in
> > > "waiting
> > > > > for
> > > > > > >>>>> correct
> > > > > > >>>>>>>>>>>> deletes to
> > > > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29,
> Adam
> > > > > > Bellemare
> > > > > > >>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I
> could
> > > > > replace
> > > > > > >> the
> > > > > > >>>>>>> second
> > > > > > >>>>>>>>>>>>>>>>>>>           repartition store
> > > > > > >>>>>>>>>>>>>>>>>>>           > and
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a
> > > > groupBy
> > > > > > and
> > > > > > >>>>>>> reduce.
> > > > > > >>>>>>>>>>>> However,
> > > > > > >>>>>>>>>>>>>>>>>>>           it looks
> > > > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store
> the
> > > > > > highwater
> > > > > > >>>>> value
> > > > > > >>>>>>>>> within
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           materialized
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> > > > > > out-of-order
> > > > > > >>>>>>> records
> > > > > > >>>>>>>>>>>> (assuming
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> my
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...).
> This
> > > in
> > > > > > effect
> > > > > > >> is
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>> same
> > > > > > >>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>           design I
> > > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two
> tables
> > > > merged
> > > > > > >>>>> together.
> > > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>>> -- Guozhang
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>> -- Guozhang
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>>
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi John & Guozhang

@John & @Guozhang Wang <wa...@gmail.com> - I have cleaned up the KIP,
pruned much of what I wrote and put a simplified diagram near the top to
illustrate the workflow. I encapsulated Jan's content at the bottom of the
document. I believe it is simpler to read by far now.

@Guozhang Wang <wa...@gmail.com>:
> #1: rekey left table
>   -> source from the left upstream, send to rekey-processor to generate
combined key, and then sink to copartition topic.
Correct.

> #2: first-join with right table
>   -> source from the right table upstream, materialize the right table.
>   -> source from the co-partition topic, materialize the rekeyed left
table, join with the right table, rekey back, and then sink to the
rekeyed-back topic.
Almost - I cleared up the KIP. We do not rekey back yet, as I need the
Foreign-Key value generated in #1 above to compare in the resolution stage.

> #3: second join
>    -> source from the rekeyed-back topic, materialize the rekeyed back
table.
>   -> source from the left upstream, materialize the left table, join with
the rekeyed back table.
Almost - As each event comes in, we just run it through a stateful
processor that checks the original ("This") KTable for the key. The value
payload then has the foreignKeyExtractor applied again as in Part #1 above,
and gets the current foreign key. Then we compare it to the joined event
that we are currently resolving. If they have the same foreign-key,
propagate the result out. If they don't, throw the event away.

The end result is that we do need to materialize 2 additional tables
(left/this-combinedkey table, and the final Joined table) as I've
illustrated in the updated KIP. I hope the diagram clears it up a lot
better. Please let me know.

Thanks again
Adam




On Sun, Dec 16, 2018 at 3:18 PM Guozhang Wang <wa...@gmail.com> wrote:

> John,
>
> Thanks a lot for the suggestions on refactoring the wiki, I agree with you
> that we should consider the KIP proposal to be easily understood by anyone
> in the future to read, and hence should provide a good summary on the
> user-facing interfaces, as well as rejected alternatives to represent
> briefly "how we came a long way to this conclusion, and what we have
> argued, disagreed, and agreed about, etc" so that readers do not need to
> dig into the DISCUSS thread to get all the details. We can, of course, keep
> the implementation details like "workflows" on the wiki page as a addendum
> section since it also has correlations.
>
> Regarding your proposal on comment 6): that's a very interesting idea! Just
> to clarify that I understands it fully correctly: the proposal's resulted
> topology is still the same as the current proposal, where we will have 3
> sub-topologies for this operator:
>
> #1: rekey left table
>    -> source from the left upstream, send to rekey-processor to generate
> combined key, and then sink to copartition topic.
>
> #2: first-join with right table
>    -> source from the right table upstream, materialize the right table.
>    -> source from the co-partition topic, materialize the rekeyed left
> table, join with the right table, rekey back, and then sink to the
> rekeyed-back topic.
>
> #3: second join
>    -> source from the rekeyed-back topic, materialize the rekeyed back
> table.
>    -> source from the left upstream, materialize the left table, join with
> the rekeyed back table.
>
> Sub-topology #1 and #3 may be merged to a single sub-topology since both of
> them read from the left table source stream. In this workflow, we need to
> materialize 4 tables (left table in #3, right table in #2, rekeyed left
> table in #2, rekeyed-back table in #3), and 2 repartition topics
> (copartition topic, rekeyed-back topic).
>
> Compared with Adam's current proposal in the workflow overview, it has the
> same num.materialize tables (left table, rekeyed left table, right table,
> out-of-ordering resolver table), and same num.internal topics (two). The
> advantage is that on the copartition topic, we can save bandwidth by not
> sending value, and in #2 the rekeyed left table is smaller since we do not
> have any values to materialize. Is that right?
>
>
> Guozhang
>
>
>
> On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io> wrote:
>
> > Hi Adam,
> >
> > Given that the committers are all pretty busy right now, I think that it
> > would help if you were to refactor the KIP a little to reduce the
> workload
> > for reviewers.
> >
> > I'd recommend the following changes:
> > * relocate all internal details to a section at the end called something
> > like "Implementation Notes" or something like that.
> > * rewrite the rest of the KIP to be a succinct as possible and mention
> only
> > publicly-facing API changes.
> > ** for example, the interface that you've already listed there, as well
> as
> > a textual description of the guarantees we'll be providing (join result
> is
> > copartitioned with the LHS, and the join result is guaranteed correct)
> >
> > A good target would be that the whole main body of the KIP, including
> > Status, Motivation, Proposal, Justification, and Rejected Alternatives
> all
> > fit "above the fold" (i.e., all fit on the screen at a comfortable zoom
> > level).
> > I think the only real Rejected Alternative that bears mention at this
> point
> > is KScatteredTable, which you could just include the executive summary on
> > (no implementation details), and link to extra details in the
> > Implementation Notes section.
> >
> > Taking a look at the wiki page, ~90% of the text there is internal
> detail,
> > which is useful for the dubious, but doesn't need to be ratified in a
> vote
> > (and would be subject to change without notice in the future anyway).
> > There's also a lot of conflicting discussion, as you've very respectfully
> > tried to preserve the original proposal from Jan while adding your own.
> > Isolating all this information in a dedicated section at the bottom frees
> > the voters up to focus on the public API part of the proposal, which is
> > really all they need to consider.
> >
> > Plus, it'll be clear to future readers which parts of the document are
> > enduring, and which parts are a snapshot of our implementation thinking
> at
> > the time.
> >
> > I'm suggesting this because I suspect that the others haven't made time
> to
> > review it partly because it seems daunting. If it seems like it would be
> a
> > huge time investment to review, people will just keep putting it off. But
> > if the KIP is a single page, then they'll be more inclined to give it a
> > read.
> >
> > Honestly, I don't think the KIP itself is that controversial (apart from
> > the scattered table thing (sorry, Jan) ). Most of the discussion has been
> > around the implementation, which we can continue more effectively in a PR
> > once the KIP has passed.
> >
> > How does that sound?
> > -John
> >
> > On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <adam.bellemare@gmail.com
> >
> > wrote:
> >
> > > 1) I believe that the resolution mechanism John has proposed is
> > sufficient
> > > - it is clean and easy and doesn't require additional RocksDB stores,
> > which
> > > reduces the footprint greatly. I don't think we need to resolve based
> on
> > > timestamp or offset anymore, but if we decide to do to that would be
> > within
> > > the bounds of the existing API.
> > >
> > > 2) Is the current API sufficient, or does it need to be altered to go
> > back
> > > to vote?
> > >
> > > 3) KScatteredTable implementation can always be added in a future
> > revision.
> > > This API does not rule it out. This implementation of this function
> would
> > > simply be replaced with `KScatteredTable.resolve()` while still
> > maintaining
> > > the existing API, thereby giving both features as Jan outlined earlier.
> > > Would this work?
> > >
> > >
> > > Thanks Guozhang, John and Jan
> > >
> > >
> > >
> > >
> > > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io>
> wrote:
> > >
> > > > Hi, all,
> > > >
> > > > >> In fact, we
> > > > >> can just keep a single final-result store with timestamps and
> reject
> > > > values
> > > > >> that have a smaller timestamp, is that right?
> > > >
> > > > > Which is the correct output should at least be decided on the
> offset
> > of
> > > > > the original message.
> > > >
> > > > Thanks for this point, Jan.
> > > >
> > > > KIP-258 is merely to allow embedding the record timestamp  in the k/v
> > > > store,
> > > > as well as providing a storage-format upgrade path.
> > > >
> > > > I might have missed it, but I think we have yet to discuss whether
> it's
> > > > safe
> > > > or desirable just to swap topic-ordering our for timestamp-ordering.
> > This
> > > > is
> > > > a very deep topic, and I think it would only pollute the current
> > > > discussion.
> > > >
> > > > What Adam has proposed is safe, given the *current* ordering
> semantics
> > > > of the system. If we can agree on his proposal, I think we can merge
> > the
> > > > feature well before the conversation about timestamp ordering even
> > takes
> > > > place, much less reaches a conclusion. In the mean time, it would
> seem
> > to
> > > > be unfortunate to have one join operator with different ordering
> > > semantics
> > > > from every other KTable operator.
> > > >
> > > > If and when that timestamp discussion takes place, many (all?) KTable
> > > > operations
> > > > will need to be updated, rendering the many:one join a small marginal
> > > cost.
> > > >
> > > > And, just to plug it again, I proposed an algorithm above that I
> > believe
> > > > provides
> > > > correct ordering without any additional metadata, and regardless of
> the
> > > > ordering semantics. I didn't bring it up further, because I felt the
> > KIP
> > > > only needs
> > > > to agree on the public API, and we can discuss the implementation at
> > > > leisure in
> > > > a PR...
> > > >
> > > > Thanks,
> > > > -John
> > > >
> > > >
> > > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <
> Jan.Filipiak@trivago.com
> > >
> > > > wrote:
> > > >
> > > > >
> > > > >
> > > > > On 10.12.2018 07:42, Guozhang Wang wrote:
> > > > > > Hello Adam / Jan / John,
> > > > > >
> > > > > > Sorry for being late on this thread! I've finally got some time
> > this
> > > > > > weekend to cleanup a load of tasks on my queue (actually I've
> also
> > > > > realized
> > > > > > there are a bunch of other things I need to enqueue while
> cleaning
> > > them
> > > > > up
> > > > > > --- sth I need to improve on my side). So here are my thoughts:
> > > > > >
> > > > > > Regarding the APIs: I like the current written API in the KIP.
> More
> > > > > > generally I'd prefer to keep the 1) one-to-many join
> > functionalities
> > > as
> > > > > > well as 2) other join types than inner as separate KIPs since 1)
> > may
> > > > > worth
> > > > > > a general API refactoring that can benefit not only foreignkey
> > joins
> > > > but
> > > > > > collocate joins as well (e.g. an extended proposal of
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > > ),
> > > > > > and I'm not sure if other join types would actually be needed
> > (maybe
> > > > left
> > > > > > join still makes sense), so it's better to
> > > > wait-for-people-to-ask-and-add
> > > > > > than add-sth-that-no-one-uses.
> > > > > >
> > > > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > > > > > KScatteredTable for users to inject their own optimization: I'd
> > > prefer
> > > > to
> > > > > > do the current option as-is, and my main rationale is for
> > > optimization
> > > > > > rooms inside the Streams internals and the API succinctness. For
> > > > advanced
> > > > > > users who may indeed prefer KScatteredTable and do their own
> > > > > optimization,
> > > > > > while it is too much of the work to use Processor API directly, I
> > > think
> > > > > we
> > > > > > can still extend the current API to support it in the future if
> it
> > > > > becomes
> > > > > > necessary.
> > > > >
> > > > > no internal optimization potential. it's a myth
> > > > >
> > > > > ¯\_(ツ)_/¯
> > > > >
> > > > > :-)
> > > > >
> > > > > >
> > > > > > Another note about step 4) resolving out-of-ordering data, as I
> > > > mentioned
> > > > > > before I think with KIP-258 (embedded timestamp with key-value
> > store)
> > > > we
> > > > > > can actually make this step simpler than the current proposal. In
> > > fact,
> > > > > we
> > > > > > can just keep a single final-result store with timestamps and
> > reject
> > > > > values
> > > > > > that have a smaller timestamp, is that right?
> > > > >
> > > > > Which is the correct output should at least be decided on the
> offset
> > of
> > > > > the original message.
> > > > >
> > > > > >
> > > > > >
> > > > > > That's all I have in mind now. Again, great appreciation to Adam
> to
> > > > make
> > > > > > such HUGE progress on this KIP!
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> > > Jan.Filipiak@trivago.com>
> > > > > > wrote:
> > > > > >
> > > > > >> If they don't find the time:
> > > > > >> They usually take the opposite path from me :D
> > > > > >> so the answer would be clear.
> > > > > >>
> > > > > >> hence my suggestion to vote.
> > > > > >>
> > > > > >>
> > > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> > > > > >>> Hi Guozhang and Matthias
> > > > > >>>
> > > > > >>> I know both of you are quite busy, but we've gotten this KIP
> to a
> > > > point
> > > > > >>> where we need more guidance on the API (perhaps a bit of a
> > > > tie-breaker,
> > > > > >> if
> > > > > >>> you will). If you have anyone else you may think should look at
> > > this,
> > > > > >>> please tag them accordingly.
> > > > > >>>
> > > > > >>> The scenario is as such:
> > > > > >>>
> > > > > >>> Current Option:
> > > > > >>> API:
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
> > partition
> > > > with
> > > > > >> the
> > > > > >>> foreignKey (repartition 1)
> > > > > >>> 2) Join the data
> > > > > >>> 3) Shuffle the data back to the original node (repartition 2)
> > > > > >>> 4) Resolve out-of-order arrival / race condition due to
> > foreign-key
> > > > > >> changes.
> > > > > >>>
> > > > > >>> Alternate Option:
> > > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
> > > > > >>> - It would be keyed on a wrapped key function: <CombinedKey<KO,
> > K>,
> > > > VR>
> > > > > >> (KO
> > > > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> > > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
> > otherwise a
> > > > > user
> > > > > >>> would be able to perform additional functions directly from the
> > > > > >>> KScatteredTable (TBD - currently out of scope).
> > > > > >>> - John's analysis 2-emails up is accurate as to the tradeoffs.
> > > > > >>>
> > > > > >>> Current Option is coded as-is. Alternate option is possible,
> but
> > > will
> > > > > >>> require for implementation details to be made in the API and
> some
> > > > > >> exposure
> > > > > >>> of new data structures into the API (ie: CombinedKey).
> > > > > >>>
> > > > > >>> I appreciate any insight into this.
> > > > > >>>
> > > > > >>> Thanks.
> > > > > >>>
> > > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > > > > adam.bellemare@gmail.com>
> > > > > >>> wrote:
> > > > > >>>
> > > > > >>>> Hi John
> > > > > >>>>
> > > > > >>>> Thanks for your feedback and assistance. I think your summary
> is
> > > > > >> accurate
> > > > > >>>> from my perspective. Additionally, I would like to add that
> > there
> > > > is a
> > > > > >> risk
> > > > > >>>> of inconsistent final states without performing the
> resolution.
> > > This
> > > > > is
> > > > > >> a
> > > > > >>>> major concern for me as most of the data I have dealt with is
> > > > produced
> > > > > >> by
> > > > > >>>> relational databases. We have seen a number of cases where a
> > user
> > > in
> > > > > the
> > > > > >>>> Rails UI has modified the field (foreign key), realized they
> > made
> > > a
> > > > > >>>> mistake, and then updated the field again with a new key. The
> > > events
> > > > > are
> > > > > >>>> propagated out as they are produced, and as such we have had
> > > > > real-world
> > > > > >>>> cases where these inconsistencies were propagated downstream
> as
> > > the
> > > > > >> final
> > > > > >>>> values due to the race conditions in the fanout of the data.
> > > > > >>>>
> > > > > >>>> This solution that I propose values correctness of the final
> > > result
> > > > > over
> > > > > >>>> other factors.
> > > > > >>>>
> > > > > >>>> We could always move this function over to using a
> > KScatteredTable
> > > > > >>>> implementation in the future, and simply deprecate it this
> join
> > > API
> > > > in
> > > > > >>>> time. I think I would like to hear more from some of the other
> > > major
> > > > > >>>> committers on which course of action they would think is best
> > > before
> > > > > any
> > > > > >>>> more coding is done.
> > > > > >>>>
> > > > > >>>> Thanks again
> > > > > >>>>
> > > > > >>>> Adam
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <
> john@confluent.io>
> > > > > wrote:
> > > > > >>>>
> > > > > >>>>> Hi Jan and Adam,
> > > > > >>>>>
> > > > > >>>>> Wow, thanks for doing that test, Adam. Those results are
> > > > encouraging.
> > > > > >>>>>
> > > > > >>>>> Thanks for your performance experience as well, Jan. I agree
> > that
> > > > > >> avoiding
> > > > > >>>>> unnecessary join outputs is especially important when the
> > fan-out
> > > > is
> > > > > so
> > > > > >>>>> high. I suppose this could also be built into the
> > implementation
> > > > > we're
> > > > > >>>>> discussing, but it wouldn't have to be specified in the KIP
> > > (since
> > > > > >> it's an
> > > > > >>>>> API-transparent optimization).
> > > > > >>>>>
> > > > > >>>>> As far as whether or not to re-repartition the data, I didn't
> > > bring
> > > > > it
> > > > > >> up
> > > > > >>>>> because it sounded like the two of you agreed to leave the
> KIP
> > > > as-is,
> > > > > >>>>> despite the disagreement.
> > > > > >>>>>
> > > > > >>>>> If you want my opinion, I feel like both approaches are
> > > reasonable.
> > > > > >>>>> It sounds like Jan values more the potential for developers
> to
> > > > > optimize
> > > > > >>>>> their topologies to re-use the intermediate nodes, whereas
> Adam
> > > > > places
> > > > > >>>>> more
> > > > > >>>>> value on having a single operator that people can use without
> > > extra
> > > > > >> steps
> > > > > >>>>> at the end.
> > > > > >>>>>
> > > > > >>>>> Personally, although I do find it exceptionally annoying
> when a
> > > > > >> framework
> > > > > >>>>> gets in my way when I'm trying to optimize something, it
> seems
> > > > better
> > > > > >> to
> > > > > >>>>> go
> > > > > >>>>> for a single operation.
> > > > > >>>>> * Encapsulating the internal transitions gives us significant
> > > > > latitude
> > > > > >> in
> > > > > >>>>> the implementation (for example, joining only at the end, not
> > in
> > > > the
> > > > > >>>>> middle
> > > > > >>>>> to avoid extra data copying and out-of-order resolution; how
> we
> > > > > >> represent
> > > > > >>>>> the first repartition keys (combined keys vs. value vectors),
> > > > etc.).
> > > > > >> If we
> > > > > >>>>> publish something like a KScatteredTable with the
> > > right-partitioned
> > > > > >> joined
> > > > > >>>>> data, then the API pretty much locks in the implementation as
> > > well.
> > > > > >>>>> * The API seems simpler to understand and use. I do mean
> > "seems";
> > > > if
> > > > > >>>>> anyone
> > > > > >>>>> wants to make the case that KScatteredTable is actually
> > simpler,
> > > I
> > > > > >> think
> > > > > >>>>> hypothetical usage code would help. From a relational algebra
> > > > > >> perspective,
> > > > > >>>>> it seems like KTable.join(KTable) should produce a new KTable
> > in
> > > > all
> > > > > >>>>> cases.
> > > > > >>>>> * That said, there might still be room in the API for a
> > different
> > > > > >>>>> operation
> > > > > >>>>> like what Jan has proposed to scatter a KTable, and then do
> > > things
> > > > > like
> > > > > >>>>> join, re-group, etc from there... I'm not sure; I haven't
> > thought
> > > > > >> through
> > > > > >>>>> all the consequences yet.
> > > > > >>>>>
> > > > > >>>>> This is all just my opinion after thinking over the
> discussion
> > so
> > > > > >> far...
> > > > > >>>>> -John
> > > > > >>>>>
> > > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > > > > >> adam.bellemare@gmail.com>
> > > > > >>>>> wrote:
> > > > > >>>>>
> > > > > >>>>>> Updated the PR to take into account John's feedback.
> > > > > >>>>>>
> > > > > >>>>>> I did some preliminary testing for the performance of the
> > > > > prefixScan.
> > > > > >> I
> > > > > >>>>>> have attached the file, but I will also include the text in
> > the
> > > > body
> > > > > >>>>> here
> > > > > >>>>>> for archival purposes (I am not sure what happens to
> attached
> > > > > files).
> > > > > >> I
> > > > > >>>>>> also updated the PR and the KIP accordingly.
> > > > > >>>>>>
> > > > > >>>>>> Summary: It scales exceptionally well for scanning large
> > values
> > > of
> > > > > >>>>>> records. As Jan mentioned previously, the real issue would
> be
> > > more
> > > > > >>>>> around
> > > > > >>>>>> processing the resulting records after obtaining them. For
> > > > instance,
> > > > > >> it
> > > > > >>>>>> takes approximately ~80-120 mS to flush the buffer and a
> > further
> > > > > >>>>> ~35-85mS
> > > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them.
> > > > Iterating
> > > > > >>>>>> through the records just to generate a simple count takes ~
> 40
> > > > times
> > > > > >>>>> longer
> > > > > >>>>>> than the flush + scan combined.
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > >>>>>> Setup:
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
> > (Xmx512m,
> > > > > >> Xms512m)
> > > > > >>>>>> CPU: i7 2.2 Ghz.
> > > > > >>>>>>
> > > > > >>>>>> Note: I am using a slightly-modified, directly-accessible
> > Kafka
> > > > > >> Streams
> > > > > >>>>>> RocksDB
> > > > > >>>>>> implementation (RocksDB.java, basically just avoiding the
> > > > > >>>>>> ProcessorContext).
> > > > > >>>>>> There are no modifications to the default RocksDB values
> > > provided
> > > > in
> > > > > >> the
> > > > > >>>>>> 2.1/trunk release.
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>> keysize = 128 bytes
> > > > > >>>>>> valsize = 512 bytes
> > > > > >>>>>>
> > > > > >>>>>> Step 1:
> > > > > >>>>>> Write X positive matching events: (key = prefix +
> left-padded
> > > > > >>>>>> auto-incrementing integer)
> > > > > >>>>>> Step 2:
> > > > > >>>>>> Write 10X negative matching events (key = left-padded
> > > > > >> auto-incrementing
> > > > > >>>>>> integer)
> > > > > >>>>>> Step 3:
> > > > > >>>>>> Perform flush
> > > > > >>>>>> Step 4:
> > > > > >>>>>> Perform prefixScan
> > > > > >>>>>> Step 5:
> > > > > >>>>>> Iterate through return Iterator and validate the count of
> > > expected
> > > > > >>>>> events.
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > >>>>>> Results:
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> ============================================================================================
> > > > > >>>>>> X = 1k (11k events total)
> > > > > >>>>>> Flush Time = 39 mS
> > > > > >>>>>> Scan Time = 7 mS
> > > > > >>>>>> 6.9 MB disk
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > >>>>>> X = 10k (110k events total)
> > > > > >>>>>> Flush Time = 45 mS
> > > > > >>>>>> Scan Time = 8 mS
> > > > > >>>>>> 127 MB
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > >>>>>> X = 100k (1.1M events total)
> > > > > >>>>>> Test1:
> > > > > >>>>>> Flush Time = 60 mS
> > > > > >>>>>> Scan Time = 12 mS
> > > > > >>>>>> 678 MB
> > > > > >>>>>>
> > > > > >>>>>> Test2:
> > > > > >>>>>> Flush Time = 45 mS
> > > > > >>>>>> Scan Time = 7 mS
> > > > > >>>>>> 576 MB
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > >>>>>> X = 1MB (11M events total)
> > > > > >>>>>> Test1:
> > > > > >>>>>> Flush Time = 52 mS
> > > > > >>>>>> Scan Time = 19 mS
> > > > > >>>>>> 7.2 GB
> > > > > >>>>>>
> > > > > >>>>>> Test2:
> > > > > >>>>>> Flush Time = 84 mS
> > > > > >>>>>> Scan Time = 34 mS
> > > > > >>>>>> 9.1 GB
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > >>>>>> X = 2.5M (27.5M events total)
> > > > > >>>>>> Test1:
> > > > > >>>>>> Flush Time = 82 mS
> > > > > >>>>>> Scan Time = 63 mS
> > > > > >>>>>> 17GB - 276 sst files
> > > > > >>>>>>
> > > > > >>>>>> Test2:
> > > > > >>>>>> Flush Time = 116 mS
> > > > > >>>>>> Scan Time = 35 mS
> > > > > >>>>>> 23GB - 361 sst files
> > > > > >>>>>>
> > > > > >>>>>> Test3:
> > > > > >>>>>> Flush Time = 103 mS
> > > > > >>>>>> Scan Time = 82 mS
> > > > > >>>>>> 19 GB - 300 sst files
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > > >>>>>>
> > > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M events. I
> > > tried
> > > > > to
> > > > > >> go
> > > > > >>>>>> to X = 10M (110M events) but RocksDB was going into the
> 100GB+
> > > > range
> > > > > >>>>> and my
> > > > > >>>>>> laptop ran out of disk. More extensive testing could be done
> > > but I
> > > > > >>>>> suspect
> > > > > >>>>>> that it would be in line with what we're seeing in the
> results
> > > > > above.
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>> At this point in time, I think the only major discussion
> point
> > > is
> > > > > >> really
> > > > > >>>>>> around what Jan and I have disagreed on: repartitioning
> back +
> > > > > >> resolving
> > > > > >>>>>> potential out of order issues or leaving that up to the
> client
> > > to
> > > > > >>>>> handle.
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>> Thanks folks,
> > > > > >>>>>>
> > > > > >>>>>> Adam
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > > > > Jan.Filipiak@trivago.com
> > > > > >>>
> > > > > >>>>>> wrote:
> > > > > >>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > > > > >>>>>>>> Hi all,
> > > > > >>>>>>>>
> > > > > >>>>>>>> Sorry that this discussion petered out... I think the 2.1
> > > > release
> > > > > >>>>>>> caused an
> > > > > >>>>>>>> extended distraction that pushed it off everyone's radar
> > > (which
> > > > > was
> > > > > >>>>>>>> precisely Adam's concern). Personally, I've also had some
> > > extend
> > > > > >>>>>>>> distractions of my own that kept (and continue to keep) me
> > > > > >>>>> preoccupied.
> > > > > >>>>>>>>
> > > > > >>>>>>>> However, calling for a vote did wake me up, so I guess Jan
> > was
> > > > on
> > > > > >> the
> > > > > >>>>>>> right
> > > > > >>>>>>>> track!
> > > > > >>>>>>>>
> > > > > >>>>>>>> I've gone back and reviewed the whole KIP document and the
> > > prior
> > > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> > > > > >>>>>>>>
> > > > > >>>>>>>> API Thoughts:
> > > > > >>>>>>>>
> > > > > >>>>>>>> 1. If I read the KIP right, you are proposing a
> many-to-one
> > > > join.
> > > > > >>>>> Could
> > > > > >>>>>>> we
> > > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip
> > the
> > > > > design
> > > > > >>>>>>> around
> > > > > >>>>>>>> and make it a oneToManyJoin?
> > > > > >>>>>>>>
> > > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join
> > type,
> > > > and
> > > > > it
> > > > > >>>>>>> seems
> > > > > >>>>>>>> like it might trick some people into using it for a
> > one-to-one
> > > > > join.
> > > > > >>>>>>> This
> > > > > >>>>>>>> would work, of course, but it would be super inefficient
> > > > compared
> > > > > to
> > > > > >>>>> a
> > > > > >>>>>>>> simple rekey-and-join.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 2. I might have missed it, but I don't think it's
> specified
> > > > > whether
> > > > > >>>>>>> it's an
> > > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
> > > > > >>>>> (neglecting
> > > > > >>>>>>> IQ),
> > > > > >>>>>>>> the rest can be achieved by filtering or by handling it in
> > the
> > > > > >>>>>>> ValueJoiner.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite
> > right.
> > > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
> > paradigms
> > > in
> > > > > >>>>> play in
> > > > > >>>>>>>> the Streams API, so it's confusing, but instead of three
> > > > > Serialized
> > > > > >>>>>>> args, I
> > > > > >>>>>>>> think it would be better to have one that allows
> > (optionally)
> > > > > >> setting
> > > > > >>>>>>> the 4
> > > > > >>>>>>>> incoming serdes. The result serde is defined by the
> > > > Materialized.
> > > > > >> The
> > > > > >>>>>>>> incoming serdes can be optional because they might already
> > be
> > > > > >>>>> available
> > > > > >>>>>>> on
> > > > > >>>>>>>> the source KTables, or the default serdes from the config
> > > might
> > > > be
> > > > > >>>>>>>> applicable.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins
> > don't
> > > > > allow
> > > > > >>>>>>> setting
> > > > > >>>>>>>> one, and it seems like it might actually be harmful, since
> > the
> > > > > rekey
> > > > > >>>>>>>> operation needs to produce results that are co-partitioned
> > > with
> > > > > the
> > > > > >>>>>>> "other"
> > > > > >>>>>>>> KTable.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't
> > > > actually
> > > > > >>>>>>> follow
> > > > > >>>>>>>> what Matthias meant about namespacing requiring
> > > "deserializing"
> > > > > the
> > > > > >>>>>>> record
> > > > > >>>>>>>> header. The headers are already Strings, so I don't think
> > that
> > > > > >>>>>>>> deserialization is required. If we applied the namespace
> at
> > > > source
> > > > > >>>>> nodes
> > > > > >>>>>>>> and stripped it at sink nodes, this would be practically
> no
> > > > > >> overhead.
> > > > > >>>>>>> The
> > > > > >>>>>>>> advantage of the namespace idea is that no public API
> change
> > > wrt
> > > > > >>>>> headers
> > > > > >>>>>>>> needs to happen, and no restrictions need to be placed on
> > > users'
> > > > > >>>>>>> headers.
> > > > > >>>>>>>>
> > > > > >>>>>>>> (Although I'm wondering if we can get away without the
> > header
> > > at
> > > > > >>>>> all...
> > > > > >>>>>>>> stay tuned)
> > > > > >>>>>>>>
> > > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM table
> > > > growing
> > > > > >>>>>>> without
> > > > > >>>>>>>> bound. As I read it, the HWM table is effectively
> > implementing
> > > > OCC
> > > > > >> to
> > > > > >>>>>>>> resolve the problem you noted with disordering when the
> > rekey
> > > is
> > > > > >>>>>>>> reversed... particularly notable when the FK changes. As
> > such,
> > > > it
> > > > > >>>>> only
> > > > > >>>>>>>> needs to track the most recent "version" (the offset in
> the
> > > > source
> > > > > >>>>>>>> partition) of each key. Therefore, it should have the same
> > > > number
> > > > > of
> > > > > >>>>>>> keys
> > > > > >>>>>>>> as the source table at all times.
> > > > > >>>>>>>>
> > > > > >>>>>>>> I see that you are aware of KIP-258, which I think might
> be
> > > > > relevant
> > > > > >>>>> in
> > > > > >>>>>>> a
> > > > > >>>>>>>> couple of ways. One: it's just about storing the timestamp
> > in
> > > > the
> > > > > >>>>> state
> > > > > >>>>>>>> store, but the ultimate idea is to effectively use the
> > > timestamp
> > > > > as
> > > > > >>>>> an
> > > > > >>>>>>> OCC
> > > > > >>>>>>>> "version" to drop disordered updates. You wouldn't want to
> > use
> > > > the
> > > > > >>>>>>>> timestamp for this operation, but if you were to use a
> > similar
> > > > > >>>>>>> mechanism to
> > > > > >>>>>>>> store the source offset in the store alongside the
> re-keyed
> > > > > values,
> > > > > >>>>> then
> > > > > >>>>>>>> you could avoid a separate table.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 6. You and Jan have been thinking about this for a long
> > time,
> > > so
> > > > > >> I've
> > > > > >>>>>>>> probably missed something here, but I'm wondering if we
> can
> > > > avoid
> > > > > >> the
> > > > > >>>>>>> HWM
> > > > > >>>>>>>> tracking at all and resolve out-of-order during a final
> join
> > > > > >>>>> instead...
> > > > > >>>>>>>>
> > > > > >>>>>>>> Let's say we're joining a left table (Integer K: Letter
> FK,
> > > > (other
> > > > > >>>>>>> data))
> > > > > >>>>>>>> to a right table (Letter K: (some data)).
> > > > > >>>>>>>>
> > > > > >>>>>>>> Left table:
> > > > > >>>>>>>> 1: (A, xyz)
> > > > > >>>>>>>> 2: (B, asd)
> > > > > >>>>>>>>
> > > > > >>>>>>>> Right table:
> > > > > >>>>>>>> A: EntityA
> > > > > >>>>>>>> B: EntityB
> > > > > >>>>>>>>
> > > > > >>>>>>>> We could do a rekey as you proposed with a combined key,
> but
> > > not
> > > > > >>>>>>>> propagating the value at all..
> > > > > >>>>>>>> Rekey table:
> > > > > >>>>>>>> A-1: (dummy value)
> > > > > >>>>>>>> B-2: (dummy value)
> > > > > >>>>>>>>
> > > > > >>>>>>>> Which we then join with the right table to produce:
> > > > > >>>>>>>> A-1: EntityA
> > > > > >>>>>>>> B-2: EntityB
> > > > > >>>>>>>>
> > > > > >>>>>>>> Which gets rekeyed back:
> > > > > >>>>>>>> 1: A, EntityA
> > > > > >>>>>>>> 2: B, EntityB
> > > > > >>>>>>>>
> > > > > >>>>>>>> And finally we do the actual join:
> > > > > >>>>>>>> Result table:
> > > > > >>>>>>>> 1: ((A, xyz), EntityA)
> > > > > >>>>>>>> 2: ((B, asd), EntityB)
> > > > > >>>>>>>>
> > > > > >>>>>>>> The thing is that in that last join, we have the
> opportunity
> > > to
> > > > > >>>>> compare
> > > > > >>>>>>> the
> > > > > >>>>>>>> current FK in the left table with the incoming PK of the
> > right
> > > > > >>>>> table. If
> > > > > >>>>>>>> they don't match, we just drop the event, since it must be
> > > > > outdated.
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets
> > > > updated
> > > > > >> to
> > > > > >>>>>>> (1:
> > > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether the
> > > final
> > > > > >>>>> state
> > > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the
> algorithm
> > > > above,
> > > > > >>>>> you
> > > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B,
> > xyz),
> > > > (B,
> > > > > >>>>>>>> EntityB)). It seems like this does give you enough
> > information
> > > > to
> > > > > >>>>> make
> > > > > >>>>>>> the
> > > > > >>>>>>>> right choice, regardless of disordering.
> > > > > >>>>>>>
> > > > > >>>>>>> Will check Adams patch, but this should work. As mentioned
> > > often
> > > > I
> > > > > am
> > > > > >>>>>>> not convinced on partitioning back for the user
> > automatically.
> > > I
> > > > > >> think
> > > > > >>>>>>> this is the real performance eater ;)
> > > > > >>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
> > > performance
> > > > of
> > > > > >>>>> the
> > > > > >>>>>>>> range scans when records change in the right table. You've
> > > said
> > > > > that
> > > > > >>>>>>> you've
> > > > > >>>>>>>> been using the algorithm you presented in production for a
> > > > while.
> > > > > >> Can
> > > > > >>>>>>> you
> > > > > >>>>>>>> give us a sense of the performance characteristics you've
> > > > > observed?
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>> Make it work, make it fast, make it beautiful. The topmost
> > > thing
> > > > > here
> > > > > >>>>> is
> > > > > >>>>>>> / was correctness. In practice I do not measure the
> > performance
> > > > of
> > > > > >> the
> > > > > >>>>>>> range scan. Usual cases I run this with is emitting 500k -
> > 1kk
> > > > rows
> > > > > >>>>>>> on a left hand side change. The range scan is just the work
> > you
> > > > > gotta
> > > > > >>>>>>> do, also when you pack your data into different formats,
> > > usually
> > > > > the
> > > > > >>>>>>> rocks performance is very tight to the size of the data and
> > we
> > > > > can't
> > > > > >>>>>>> really change that. It is more important for users to
> prevent
> > > > > useless
> > > > > >>>>>>> updates to begin with. My left hand side is guarded to drop
> > > > changes
> > > > > >>>>> that
> > > > > >>>>>>> are not going to change my join output.
> > > > > >>>>>>>
> > > > > >>>>>>> usually it's:
> > > > > >>>>>>>
> > > > > >>>>>>> drop unused fields and then don't forward if
> old.equals(new)
> > > > > >>>>>>>
> > > > > >>>>>>> regarding to the performance of creating an iterator for
> > > smaller
> > > > > >>>>>>> fanouts, users can still just do a group by first then
> > anyways.
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>> I could only think of one alternative, but I'm not sure if
> > > it's
> > > > > >>>>> better
> > > > > >>>>>>> or
> > > > > >>>>>>>> worse... If the first re-key only needs to preserve the
> > > original
> > > > > >> key,
> > > > > >>>>>>> as I
> > > > > >>>>>>>> proposed in #6, then we could store a vector of keys in
> the
> > > > value:
> > > > > >>>>>>>>
> > > > > >>>>>>>> Left table:
> > > > > >>>>>>>> 1: A,...
> > > > > >>>>>>>> 2: B,...
> > > > > >>>>>>>> 3: A,...
> > > > > >>>>>>>>
> > > > > >>>>>>>> Gets re-keyed:
> > > > > >>>>>>>> A: [1, 3]
> > > > > >>>>>>>> B: [2]
> > > > > >>>>>>>>
> > > > > >>>>>>>> Then, the rhs part of the join would only need a regular
> > > > > single-key
> > > > > >>>>>>> lookup.
> > > > > >>>>>>>> Of course we have to deal with the problem of large
> values,
> > as
> > > > > >>>>> there's
> > > > > >>>>>>> no
> > > > > >>>>>>>> bound on the number of lhs records that can reference rhs
> > > > records.
> > > > > >>>>>>> Offhand,
> > > > > >>>>>>>> I'd say we could page the values, so when one row is past
> > the
> > > > > >>>>>>> threshold, we
> > > > > >>>>>>>> append the key for the next page. Then in most cases, it
> > would
> > > > be
> > > > > a
> > > > > >>>>>>> single
> > > > > >>>>>>>> key lookup, but for large fan-out updates, it would be one
> > per
> > > > > (max
> > > > > >>>>>>> value
> > > > > >>>>>>>> size)/(avg lhs key size).
> > > > > >>>>>>>>
> > > > > >>>>>>>> This seems more complex, though... Plus, I think there's
> > some
> > > > > extra
> > > > > >>>>>>>> tracking we'd need to do to know when to emit a
> retraction.
> > > For
> > > > > >>>>> example,
> > > > > >>>>>>>> when record 1 is deleted, the re-key table would just have
> > (A:
> > > > > [3]).
> > > > > >>>>>>> Some
> > > > > >>>>>>>> kind of tombstone is needed so that the join result for 1
> > can
> > > > also
> > > > > >> be
> > > > > >>>>>>>> retracted.
> > > > > >>>>>>>>
> > > > > >>>>>>>> That's all!
> > > > > >>>>>>>>
> > > > > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful
> KIP.
> > > > Sorry
> > > > > >> the
> > > > > >>>>>>>> discussion has been slow.
> > > > > >>>>>>>> -John
> > > > > >>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > > > > >>>>> Jan.Filipiak@trivago.com>
> > > > > >>>>>>>> wrote:
> > > > > >>>>>>>>
> > > > > >>>>>>>>> Id say you can just call the vote.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> that happens all the time, and if something comes up, it
> > just
> > > > > goes
> > > > > >>>>> back
> > > > > >>>>>>>>> to discuss.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> would not expect to much attention with another another
> > email
> > > > in
> > > > > >>>>> this
> > > > > >>>>>>>>> thread.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> best Jan
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > > > > >>>>>>>>>> Hello Contributors
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do need
> to
> > > bump
> > > > > >>>>> this to
> > > > > >>>>>>>>> keep
> > > > > >>>>>>>>>> visibility up. I am still intending to push this through
> > > once
> > > > > >>>>>>> contributor
> > > > > >>>>>>>>>> feedback is given.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Main points that need addressing:
> > > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
> > singular
> > > > > graph
> > > > > >>>>> node
> > > > > >>>>>>>>> into
> > > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right
> > now. I
> > > > am
> > > > > a
> > > > > >>>>> bit
> > > > > >>>>>>>>> fuzzy
> > > > > >>>>>>>>>> on how the optimizations are supposed to work, so I
> would
> > > > > >>>>> appreciate
> > > > > >>>>>>> any
> > > > > >>>>>>>>>> help on this aspect.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This thread
> > has
> > > > > much
> > > > > >>>>>>>>> discourse
> > > > > >>>>>>>>>> between Jan and I between the current highwater mark
> > > proposal
> > > > > and
> > > > > >> a
> > > > > >>>>>>>>> groupBy
> > > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need to
> > > > strictly
> > > > > >>>>> handle
> > > > > >>>>>>>>> any
> > > > > >>>>>>>>>> chance of out-of-order data and leave none of it up to
> the
> > > > > >>>>> consumer.
> > > > > >>>>>>> Any
> > > > > >>>>>>>>>> comments or suggestions here would also help.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> 3) Anything else that you see that would prevent this
> from
> > > > > moving
> > > > > >>>>> to a
> > > > > >>>>>>>>> vote?
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Thanks
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Adam
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > > > > >>>>>>>>> adam.bellemare@gmail.com>
> > > > > >>>>>>>>>> wrote:
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>> Hi Jan
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> > > > > >>>>> Stores.persistentWindowStore,
> > > > > >>>>>>> you
> > > > > >>>>>>>>>>> actually only need to specify the amount of segments
> you
> > > want
> > > > > and
> > > > > >>>>> how
> > > > > >>>>>>>>> large
> > > > > >>>>>>>>>>> they are. To the best of my understanding, what happens
> > is
> > > > that
> > > > > >>>>> the
> > > > > >>>>>>>>>>> segments are automatically rolled over as new data with
> > new
> > > > > >>>>>>> timestamps
> > > > > >>>>>>>>> are
> > > > > >>>>>>>>>>> created. We use this exact functionality in some of the
> > > work
> > > > > done
> > > > > >>>>>>>>>>> internally at my company. For reference, this is the
> > > hopping
> > > > > >>>>> windowed
> > > > > >>>>>>>>> store.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> In the code that I have provided, there are going to be
> > two
> > > > 24h
> > > > > >>>>>>>>> segments.
> > > > > >>>>>>>>>>> When a record is put into the windowStore, it will be
> > > > inserted
> > > > > at
> > > > > >>>>>>> time
> > > > > >>>>>>>>> T in
> > > > > >>>>>>>>>>> both segments. The two segments will always overlap by
> > 12h.
> > > > As
> > > > > >>>>> time
> > > > > >>>>>>>>> goes on
> > > > > >>>>>>>>>>> and new records are added (say at time T+12h+), the
> > oldest
> > > > > >> segment
> > > > > >>>>>>> will
> > > > > >>>>>>>>> be
> > > > > >>>>>>>>>>> automatically deleted and a new segment created. The
> > > records
> > > > > are
> > > > > >>>>> by
> > > > > >>>>>>>>> default
> > > > > >>>>>>>>>>> inserted with the context.timestamp(), such that it is
> > the
> > > > > record
> > > > > >>>>>>> time,
> > > > > >>>>>>>>> not
> > > > > >>>>>>>>>>> the clock time, which is used.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> To the best of my understanding, the timestamps are
> > > retained
> > > > > when
> > > > > >>>>>>>>>>> restoring from the changelog.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL
> at a
> > > > > >>>>>>> segment-level,
> > > > > >>>>>>>>>>> instead of at an individual record level.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > > > > >>>>>>> Jan.Filipiak@trivago.com>
> > > > > >>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
> > > > > ClassCastException
> > > > > >>>>> or
> > > > > >>>>>>>>>>>> similar.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> You either would have to specify the window you
> > fetch/put
> > > or
> > > > > >>>>> iterate
> > > > > >>>>>>>>>>>> across all windows the key was found in right?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> I just hope the window-store doesn't check stream-time
> > > under
> > > > > the
> > > > > >>>>>>> hoods
> > > > > >>>>>>>>>>>> that would be a questionable interface.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> If it does: did you see my comment on checking all the
> > > > windows
> > > > > >>>>>>> earlier?
> > > > > >>>>>>>>>>>> that would be needed to actually give reasonable time
> > > > > gurantees.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Best
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > > > > >>>>>>>>>>>>> Hi Jan
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed
> > the
> > > > > state
> > > > > >>>>>>> store,
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>> the ProcessorSupplier.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>> Adam
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> @Guozhang
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed
> something
> > > that
> > > > > >>>>> will be
> > > > > >>>>>>>>>>>>>>> extremely
> > > > > >>>>>>>>>>>>>>> useful for this KIP.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> @Jan
> > > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I
> will
> > > not
> > > > > be
> > > > > >>>>>>> moving
> > > > > >>>>>>>>>>>> ahead
> > > > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy
> > solution
> > > > as
> > > > > >> you
> > > > > >>>>>>>>>>>> propose.
> > > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it
> yourself
> > > off
> > > > > of
> > > > > >>>>> my
> > > > > >>>>>>>>>>>> current PR
> > > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I would
> > be
> > > > more
> > > > > >>>>> than
> > > > > >>>>>>>>>>>> happy to
> > > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it
> stands
> > > > right
> > > > > >>>>> now,
> > > > > >>>>>>> I do
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>> really have more time to invest into alternatives
> > > without
> > > > > >>>>> there
> > > > > >>>>>>>>> being
> > > > > >>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>> strong indication from the binding voters which
> they
> > > > would
> > > > > >>>>>>> prefer.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up
> on
> > > the
> > > > > >>>>> streams
> > > > > >>>>>>>>> DSL
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>> some time already, otherwise I would have pulled
> this
> > > KIP
> > > > > >>>>> through
> > > > > >>>>>>>>>>>> already.
> > > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on
> > PAPI.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed
> > > state
> > > > > >>>>> store
> > > > > >>>>>>> in
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I
> will
> > > come
> > > > > >> back
> > > > > >>>>>>> for
> > > > > >>>>>>>>>>>> final
> > > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of
> the
> > > > > binding
> > > > > >>>>>>> voters
> > > > > >>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated
> it
> > > > > >>>>> according
> > > > > >>>>>>> to
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> latest plan:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed
> > store.
> > > > > This
> > > > > >>>>>>> could
> > > > > >>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they
> are
> > > > > >>>>> completed.
> > > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Adam
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier
> already
> > > > > updated
> > > > > >>>>> in
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> PR?
> > > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
> > > > something?
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > > > > >>>>>>> wangguoz@gmail.com>
> > > > > >>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the
> > > wrong
> > > > > >> link,
> > > > > >>>>>>> as it
> > > > > >>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of
> > > > KIP-258
> > > > > >>>>> we do
> > > > > >>>>>>>>>>>> want to
> > > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source
> KTable"
> > > such
> > > > > >> that
> > > > > >>>>>>>>>>>> instead of
> > > > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized
> store,
> > > > i.e.
> > > > > >>>>>>> following
> > > > > >>>>>>>>>>>>>>>> offset
> > > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older
> than
> > > the
> > > > > >>>>> current
> > > > > >>>>>>>>>>>> key's
> > > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Guozhang
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> > > > > >>>>>>>>> wangguoz@gmail.com>
> > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Hello Adam,
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final
> > step
> > > > > (i.e.
> > > > > >>>>> the
> > > > > >>>>>>>>> high
> > > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with
> a
> > > > window
> > > > > >>>>>>> store),
> > > > > >>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>> think
> > > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > >>>>>>>>>>>>>>>>>
> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value
> > > store
> > > > > >>>>> (i.e.
> > > > > >>>>>>> only
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage,
> as
> > > > > >>>>> described
> > > > > >>>>>>> in
> > > > > >>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533
> ,
> > is
> > > > > that
> > > > > >>>>> we
> > > > > >>>>>>> can
> > > > > >>>>>>>>>>>> then
> > > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
> > > > timestamp
> > > > > is
> > > > > >>>>>>>>> smaller
> > > > > >>>>>>>>>>>> than
> > > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I
> think
> > it
> > > > is
> > > > > >>>>> very
> > > > > >>>>>>>>>>>> similar to
> > > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
> > > > filtering,
> > > > > >>>>> while
> > > > > >>>>>>>>> you
> > > > > >>>>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the
> > joining
> > > > > >> records
> > > > > >>>>>>> are
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> correctly
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the final
> > > stage.
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and
> hence
> > > > > >>>>>>> non-windowed
> > > > > >>>>>>>>>>>> KTables
> > > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really
> > have
> > > a
> > > > > good
> > > > > >>>>>>>>> support
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>> their joins anyways (
> > > > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > > > > >>>>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
> > KTable-KTable
> > > > > >>>>> non-key
> > > > > >>>>>>>>> joins
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> Guozhang
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would
> grow
> > > > > >> endlessly
> > > > > >>>>>>> based
> > > > > >>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of
> > > (<this
> > > > > >>>>> table
> > > > > >>>>>>>>>>>> primary
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> key>,
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used
> > to
> > > > > >>>>>>> differentiate
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> between
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal
> > > would
> > > > > be
> > > > > >>>>> to
> > > > > >>>>>>>>>>>> replace
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This
> > would
> > > > > allow
> > > > > >>>>> the
> > > > > >>>>>>>>> same
> > > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This
> > > > should
> > > > > >>>>> allow
> > > > > >>>>>>> for
> > > > > >>>>>>>>>>>> all
> > > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and
> should
> > be
> > > > > >>>>>>> customizable
> > > > > >>>>>>>>>>>> by
> > > > > >>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps
> > just
> > > > 10
> > > > > >>>>>>> minutes
> > > > > >>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> window,
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the
> > > trick
> > > > > >> here.
> > > > > >>>>>>> Even
> > > > > >>>>>>>>>>>> if I
> > > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
> > repartitioning
> > > > > >>>>> optional
> > > > > >>>>>>>>>>>> since I
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a
> > > little
> > > > > bit
> > > > > >>>>>>>>>>>> sceptical
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one
> > could
> > > > run
> > > > > >>>>> into
> > > > > >>>>>>>>>>>> problems
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window border. I
> > will
> > > > > check
> > > > > >>>>> you
> > > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we
> > > could
> > > > > >>>>> still
> > > > > >>>>>>>>> check
> > > > > >>>>>>>>>>>>>>>>>> _all_
> > > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance
> > impact I
> > > > > >> guess.
> > > > > >>>>>>> Will
> > > > > >>>>>>>>>>>> let
> > > > > >>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct as
> > is. I
> > > > > >>>>> wouldn't
> > > > > >>>>>>> not
> > > > > >>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> > timestamp(A)  <
> > > > > >>>>>>>>> timestamp(B).
> > > > > >>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> think
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> we can't expect that.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> @Jan
> > > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now -
> thanks
> > > for
> > > > > the
> > > > > >>>>>>>>>>>> diagram, it
> > > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not
> > have
> > > > the
> > > > > >>>>>>> original
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> primary
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
> > available
> > > > > then
> > > > > >>>>> you
> > > > > >>>>>>>>>>>> would be
> > > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map.
> That
> > > > being
> > > > > >>>>> said,
> > > > > >>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> encourage
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for
> > clarity
> > > > for
> > > > > >>>>>>> everyone
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> else.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard
> > > work.
> > > > > But
> > > > > >>>>> I
> > > > > >>>>>>>>>>>> understand
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> > original
> > > > > >> primary
> > > > > >>>>>>> key,
> > > > > >>>>>>>>> We
> > > > > >>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI
> and
> > > > > >> basically
> > > > > >>>>>>> not
> > > > > >>>>>>>>>>>> using
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed
> that
> > in
> > > > > >>>>> original
> > > > > >>>>>>> DSL
> > > > > >>>>>>>>>>>> its
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on
> > my
> > > > end.
> > > > > >>>>> Will
> > > > > >>>>>>>>>>>> finish
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map
> > stay
> > > > > >> inside
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> State
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes
> have
> > > > > >>>>> propagated?
> > > > > >>>>>>>>> Isn't
> > > > > >>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state
> > > store?
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor
> is
> > > > gonna
> > > > > >>>>>>> return
> > > > > >>>>>>>>>>>> `null`
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there
> is
> > > > going
> > > > > to
> > > > > >>>>> be
> > > > > >>>>>>> a
> > > > > >>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this
> store
> > > > > directly
> > > > > >>>>> for
> > > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a
> > > regular
> > > > > >>>>> store,
> > > > > >>>>>>>>>>>> satisfying
> > > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join.
> > The
> > > > > >>>>> Windowed
> > > > > >>>>>>>>>>>> store is
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull
> > > operation
> > > > > we
> > > > > >>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have
> the
> > > > > window
> > > > > >>>>>>> store
> > > > > >>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> the values then.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom
> group
> > > by
> > > > > >>>>> before
> > > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i
> think
> > > it
> > > > > >> would
> > > > > >>>>>>> help
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> users
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the
> > > original
> > > > > >>>>> primary
> > > > > >>>>>>>>> key
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> issue I
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid foundation
> > to
> > > > > build
> > > > > >>>>> on.
> > > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user.
> very
> > > > > >>>>>>> unfortunate. I
> > > > > >>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not
> > > think
> > > > > its
> > > > > >>>>> a
> > > > > >>>>>>> good
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> decision.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Thanks
> > > > > >>>>>>>>>>>>>>>>>>> Adam
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta
> > Dumbre <
> > > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > > > > >>>>>>> dumbreprajakta311@gmail.com
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan
> > > > Filipiak
> > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           wrote:
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make such a
> > > > chart.
> > > > > >> last
> > > > > >>>>>>> time i
> > > > > >>>>>>>>>>>> didn't
> > > > > >>>>>>>>>>>>>>>>>>>           get along
> > > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your
> > > > charts.
> > > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare
> > > > wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of
> what I
> > > > > >> attempted
> > > > > >>>>> on
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> KIP.
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > > > > >>>>>>>>>>>>>>>>>>>           <
> > > > > >>>>>>>>>>>>
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the
> start
> > of
> > > > my
> > > > > own
> > > > > >>>>>>>>>>>> implementation
> > > > > >>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>           this
> > > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not
> get
> > > it
> > > > to
> > > > > >>>>> work I
> > > > > >>>>>>> have
> > > > > >>>>>>>>>>>> since
> > > > > >>>>>>>>>>>>>>>>>>>           discarded the
> > > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if
> you
> > > > wish
> > > > > to
> > > > > >>>>>>> continue
> > > > > >>>>>>>>>>>> pursuing
> > > > > >>>>>>>>>>>>>>>>>>>           for your
> > > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you
> > > please
> > > > > >>>>> create a
> > > > > >>>>>>>>>>>> diagram on
> > > > > >>>>>>>>>>>>>>>>>>>           the KIP
> > > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your
> solution.
> > > > > Please
> > > > > >>>>> feel
> > > > > >>>>>>> free
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> use
> > > > > >>>>>>>>>>>>>>>>>>>           the image I
> > > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting point.
> I
> > am
> > > > > having
> > > > > >>>>>>> trouble
> > > > > >>>>>>>>>>>>>>>>>>>           understanding your
> > > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a
> > > > carefully
> > > > > >>>>>>> constructed
> > > > > >>>>>>>>>>>> diagram
> > > > > >>>>>>>>>>>>>>>>>>>           will clear
> > > > > >>>>>>>>>>>>>>>>>>>           > up
> > > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings.
> Alternately,
> > > > > please
> > > > > >>>>> post a
> > > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> > > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess
> at
> > > what
> > > > > you
> > > > > >>>>>>> mean, and
> > > > > >>>>>>>>>>>> since I
> > > > > >>>>>>>>>>>>>>>>>>>           value my
> > > > > >>>>>>>>>>>>>>>>>>>           > own
> > > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value yours,
> I
> > > > > believe
> > > > > >> it
> > > > > >>>>> is
> > > > > >>>>>>> your
> > > > > >>>>>>>>>>>>>>>>>>>           responsibility to
> > > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation instead
> > of
> > > me
> > > > > >>>>> trying to
> > > > > >>>>>>>>> guess.
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > > Adam
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM,
> Jan
> > > > > Filipiak
> > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing
> interested.
> > > > kafka
> > > > > >>>>>>> streams at
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>           point supports
> > > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as
> both
> > > > > streams
> > > > > >>>>> have
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> same
> > > > > >>>>>>>>>>>>>>>>>>> key.
> > > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a
> > > join
> > > > > >> where a
> > > > > >>>>>>> KTable
> > > > > >>>>>>>>>>>> and a
> > > > > >>>>>>>>>>>>>>>>>>>           KTable can
> > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship
> (1:n).
> > > We
> > > > > >> exploit
> > > > > >>>>>>> that
> > > > > >>>>>>>>>>>> rocksdb
> > > > > >>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted
> (At
> > > > least
> > > > > >>>>>>> exposes an
> > > > > >>>>>>>>>>>> API to
> > > > > >>>>>>>>>>>>>>>>>>>           access the
> > > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted
> fashion).
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats
> are
> > > well
> > > > > >>>>>>> understood
> > > > > >>>>>>>>> now
> > > > > >>>>>>>>>>>> and we
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > basically
> > > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API
> Design
> > (
> > > > when
> > > > > >> Adam
> > > > > >>>>>>> sees
> > > > > >>>>>>>>> my
> > > > > >>>>>>>>>>>> newest
> > > > > >>>>>>>>>>>>>>>>>>>           message).
> > > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of
> > > > loosing
> > > > > >>>>> those
> > > > > >>>>>>> kinda
> > > > > >>>>>>>>>>>>>>>>>>>           arguments within
> > > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have no
> > clue
> > > > > why.
> > > > > >> So
> > > > > >>>>> I
> > > > > >>>>>>>>>>>> literally
> > > > > >>>>>>>>>>>>>>>>>>>           can't wait for
> > > > > >>>>>>>>>>>>>>>>>>>           > you
> > > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and
> > > give
> > > > > you
> > > > > >>>>>>> opinion on
> > > > > >>>>>>>>>>>> how we
> > > > > >>>>>>>>>>>>>>>>>>>           should
> > > > > >>>>>>>>>>>>>>>>>>>           > design
> > > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
> > oneToManyJoin
> > > > and
> > > > > >> how
> > > > > >>>>>>> many
> > > > > >>>>>>>>>>>> power we
> > > > > >>>>>>>>>>>>>>>>>>>           want to give
> > > > > >>>>>>>>>>>>>>>>>>>           > to
> > > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where
> > > > > simplicity
> > > > > >>>>> isn't
> > > > > >>>>>>>>>>>> really that
> > > > > >>>>>>>>>>>>>>>>>>>           as users
> > > > > >>>>>>>>>>>>>>>>>>>           > still
> > > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on
> the
> > > > > >> discussion
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan
> > > wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I
> > found
> > > > this
> > > > > >>>>> subject
> > > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> > > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a
> join
> > > > > table of
> > > > > >>>>> two
> > > > > >>>>>>>>>>>> streams? Is
> > > > > >>>>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>           > somewhere
> > > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
> > requirement
> > > or
> > > > > >>>>> proposal?
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan
> > > > > Filipiak
> > > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
> > > Bellemare
> > > > > >> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a
> > > > > Windowed
> > > > > >>>>> Store
> > > > > >>>>>>> to
> > > > > >>>>>>>>>>>> store the
> > > > > >>>>>>>>>>>>>>>>>>>           highwater
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this
> should
> > > work
> > > > > >> fine,
> > > > > >>>>>>> with
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> caveat
> > > > > >>>>>>>>>>>>>>>>>>>           being that
> > > > > >>>>>>>>>>>>>>>>>>>           > it
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order
> > arrival
> > > > > for up
> > > > > >>>>> to
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> size of
> > > > > >>>>>>>>>>>>>>>>>>>           the window
> > > > > >>>>>>>>>>>>>>>>>>>           > (ie:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would
> > remove
> > > > the
> > > > > >>>>>>> possibility
> > > > > >>>>>>>>>>>> of it
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> being
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > unbounded
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
> > > suggestion, I
> > > > > >>>>> believe
> > > > > >>>>>>> this
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>>> where
> > > > > >>>>>>>>>>>>>>>>>>>           we will
> > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement.
> While I
> > > do
> > > > > not
> > > > > >>>>>>> disagree
> > > > > >>>>>>>>>>>> with your
> > > > > >>>>>>>>>>>>>>>>>>>           statement
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional
> > > joins
> > > > > done
> > > > > >>>>> in a
> > > > > >>>>>>>>>>>> real-world
> > > > > >>>>>>>>>>>>>>>>>>>           workflow, I
> > > > > >>>>>>>>>>>>>>>>>>>           > do
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively
> > deal
> > > > > with
> > > > > >>>>>>>>> out-of-order
> > > > > >>>>>>>>>>>>>>>>>>> arrival
> > > > > >>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent
> joins. I
> > > > have
> > > > > >>>>>>> attempted
> > > > > >>>>>>>>> what
> > > > > >>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>           think you have
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a
> high-water,
> > > > using
> > > > > >>>>>>> groupBy and
> > > > > >>>>>>>>>>>> reduce)
> > > > > >>>>>>>>>>>>>>>>>>>           and found
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too
> > > > quickly,
> > > > > or
> > > > > >>>>> the
> > > > > >>>>>>> load
> > > > > >>>>>>>>> on
> > > > > >>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>           stream thread
> > > > > >>>>>>>>>>>>>>>>>>>           > is
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages
> will
> > > > arrive
> > > > > >>>>>>>>> out-of-order
> > > > > >>>>>>>>>>>> and be
> > > > > >>>>>>>>>>>>>>>>>>>           incorrectly
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> > > > intermediate
> > > > > >>>>> event
> > > > > >>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>> represented
> > > > > >>>>>>>>>>>>>>>>>>>           as the
> > > > > >>>>>>>>>>>>>>>>>>>           > final
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on
> your
> > > > > groupBy
> > > > > >>>>>>>>>>>> implementation.
> > > > > >>>>>>>>>>>>>>>>>>>           There must be
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it
> is,
> > I
> > > > > would
> > > > > >>>>> just
> > > > > >>>>>>> like
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I
> > would
> > > > > like
> > > > > >> to
> > > > > >>>>>>> clarify
> > > > > >>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>           we draw a
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the
> scattered
> > > > events
> > > > > >>>>> back to
> > > > > >>>>>>>>> their
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> original
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I
> know
> > > how
> > > > > to
> > > > > >>>>>>>>> conclusively
> > > > > >>>>>>>>>>>> deal
> > > > > >>>>>>>>>>>>>>>>>>>           with
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a
> given
> > > time
> > > > > >> frame,
> > > > > >>>>>>> and to
> > > > > >>>>>>>>>>>> ensure
> > > > > >>>>>>>>>>>>>>>>>>>           that the
> > > > > >>>>>>>>>>>>>>>>>>>           > data
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with
> the
> > > > input
> > > > > >>>>> events.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to
> share
> > > that
> > > > > >>>>>>> illustrates
> > > > > >>>>>>>>> your
> > > > > >>>>>>>>>>>>>>>>>>>           approach, I
> > > > > >>>>>>>>>>>>>>>>>>>           > would
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would
> > remove
> > > > any
> > > > > >>>>>>>>>>>> misunderstandings
> > > > > >>>>>>>>>>>>>>>>>>>           that I may
> > > > > >>>>>>>>>>>>>>>>>>>           > have.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for
> my
> > > > code.
> > > > > I
> > > > > >>>>> don't
> > > > > >>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>           something easily
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated
> > with
> > > > > >>>>> OO-patterns.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey,
> V
> > > > > value, T
> > > > > >>>>>>>>> aggregate)
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> > > > currentStateAsMap =
> > > > > >>>>>>>>>>>> asMap(aggregate);
> > > > > >>>>>>>>>>>>>>>>>>> <<
> > > > > >>>>>>>>>>>>>>>>>>>           imaginary
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> > > > > >>>>> mapper.apply(value);
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the
> > place
> > > > > where
> > > > > >>>>> people
> > > > > >>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>>>>>>>>           gonna have
> > > > > >>>>>>>>>>>>>>>>>>>           > issues
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't
> do
> > > it.
> > > > > we
> > > > > >>>>> would
> > > > > >>>>>>> need
> > > > > >>>>>>>>>>>> to find
> > > > > >>>>>>>>>>>>>>>>>>>           a solution
> > > > > >>>>>>>>>>>>>>>>>>>           > here.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate
> the
> > > > > field in
> > > > > >>>>> the
> > > > > >>>>>>>>>>>> joiner, so
> > > > > >>>>>>>>>>>>>>>>>>>           that we can
> > > > > >>>>>>>>>>>>>>>>>>>           > pick
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate.
> Probably
> > > you
> > > > > have
> > > > > >>>>> not
> > > > > >>>>>>>>> thought
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>           this in your
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open
> > to
> > > > > find a
> > > > > >>>>>>> generic
> > > > > >>>>>>>>>>>> solution
> > > > > >>>>>>>>>>>>>>>>>>>           here. In my
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken
> in
> > > > > >>>>>>>>> KTableImpl.GroupBy
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>           looses
> > > > > >>>>>>>>>>>>>>>>>>>           > the keys
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the
> aggregate
> > > > key.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted
> it
> > > away
> > > > > back
> > > > > >>>>>>> then way
> > > > > >>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>> i
> > > > > >>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>           > thinking
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is
> why I
> > > > > didn't
> > > > > >>>>>>> realize
> > > > > >>>>>>>>> its
> > > > > >>>>>>>>>>>>>>>>>>>           significance here.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > currentStateAsMap.put(mapper.apply(m),
> > > > > >> m);
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > currentStateAsMap.put(toModifyKey,
> > > > > >> value);
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > currentStateAsMap.remove(toModifyKey);
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> if(currentStateAsMap.isEmpty()){
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> > > > > >>>>>>> asAggregateType(currentStateAsMap)
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35
> PM,
> > > Jan
> > > > > >>>>> Filipiak
> > > > > >>>>>>> <
> > > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
> > > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing
> > Matthias
> > > > to
> > > > > >>>>> speed!
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I
> think
> > > > > >> re-keying
> > > > > >>>>>>> back
> > > > > >>>>>>>>>>>> should be
> > > > > >>>>>>>>>>>>>>>>>>>           optional at
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> > > > > KScatteredTable
> > > > > >>>>> with
> > > > > >>>>>>>>>>>> reshuffle()
> > > > > >>>>>>>>>>>>>>>>>>>           returning
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined>
> to
> > > make
> > > > > the
> > > > > >>>>>>> backwards
> > > > > >>>>>>>>>>>>>>>>>>>           repartitioning
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of
> > > doing
> > > > > the
> > > > > >>>>> out
> > > > > >>>>>>> of
> > > > > >>>>>>>>> order
> > > > > >>>>>>>>>>>>>>>>>>>           processing using
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark
> > > > tracking.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded
> growth
> > is
> > > > > just
> > > > > >>>>> scary
> > > > > >>>>>>> + It
> > > > > >>>>>>>>>>>> saves
> > > > > >>>>>>>>>>>>>>>>>>> us
> > > > > >>>>>>>>>>>>>>>>>>>           the header
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of
> > > always
> > > > > >>>>>>> repartitioning
> > > > > >>>>>>>>>>>> back is
> > > > > >>>>>>>>>>>>>>>>>>>           just not so
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has
> been
> > > > done
> > > > > >>>>> before
> > > > > >>>>>>> we
> > > > > >>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>           back and
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards
> > is
> > > > > really
> > > > > >>>>>>> common.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
> > > > Bellemare
> > > > > >>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your
> feedback,
> > I
> > > do
> > > > > >>>>>>> appreciate
> > > > > >>>>>>>>> it!
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be
> > > > > possible,
> > > > > >> it
> > > > > >>>>>>> would
> > > > > >>>>>>>>>>>> require
> > > > > >>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           > deserialize
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies
> a
> > > > > runtime
> > > > > >>>>>>> overhead.
> > > > > >>>>>>>>> I
> > > > > >>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid
> > the
> > > > > >>>>> overhead.
> > > > > >>>>>>> If
> > > > > >>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still
> add
> > > > name
> > > > > >>>>> spacing
> > > > > >>>>>>>>> later
> > > > > >>>>>>>>>>>> on.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with
> > using a
> > > > > >> reserved
> > > > > >>>>>>> string
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>           document it.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the
> > > design
> > > > it
> > > > > >> the
> > > > > >>>>>>> type of
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
> > > > correctly,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have
> > > table1
> > > > > and
> > > > > >>>>> table2
> > > > > >>>>>>>>>>>> swapped.
> > > > > >>>>>>>>>>>>>>>>>>>           Here is how it
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records
> > that
> > > > > contain
> > > > > >>>>> the
> > > > > >>>>>>>>>>>> foreign key
> > > > > >>>>>>>>>>>>>>>>>>>           within their
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> > > > > <a,(fk=A,bar=1)>,
> > > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>,
> > > <B,Y>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is
> required
> > > to
> > > > > >> extract
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> foreign
> > > > > >>>>>>>>>>>>>>>>>>> key.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper:
> (
> > > > value
> > > > > =>
> > > > > >>>>>>> value.fk
> > > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to
> each
> > > > > element
> > > > > >> in
> > > > > >>>>>>>>> table1,
> > > > > >>>>>>>>>>>> and a
> > > > > >>>>>>>>>>>>>>>>>>>           new combined
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> > > > (fk=A,bar=1)>,
> > > > > >>>>> <A-b,
> > > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > > > > >>>>>>>>>>>>>>>>>>>           <B-c,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> > > > > >> copartitioned
> > > > > >>>>>>> with
> > > > > >>>>>>>>>>>> table2:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
> > Partition
> > > > 0:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
> > > > > >>>>> (fk=A,bar=1)>,
> > > > > >>>>>>> <A-b,
> > > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
> > Partition
> > > > 1:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
> > > > > >> (fk=B,bar=3)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be
> > > joined
> > > > > >>>>> together
> > > > > >>>>>>>>> locally
> > > > > >>>>>>>>>>>> by
> > > > > >>>>>>>>>>>>>>>>>>>           applying the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design
> > and
> > > > my
> > > > > >>>>> design
> > > > > >>>>>>>>>>>> deviate. My
> > > > > >>>>>>>>>>>>>>>>>>>           design goes
> > > > > >>>>>>>>>>>>>>>>>>>           > on
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
> > post-join
> > > > and
> > > > > >>>>> resolve
> > > > > >>>>>>>>>>>> out-of-order
> > > > > >>>>>>>>>>>>>>>>>>>           arrival of
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data
> > > keyed
> > > > > just
> > > > > >>>>> the
> > > > > >>>>>>>>>>>> original key.
> > > > > >>>>>>>>>>>>>>>>>>>           I do not
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the
> > > > internals
> > > > > >>>>>>> outside of
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make
> for
> > > > larger
> > > > > >>>>>>> footprint,
> > > > > >>>>>>>>>>>> but it
> > > > > >>>>>>>>>>>>>>>>>>>           removes all
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order
> > > > arrivals
> > > > > >> and
> > > > > >>>>>>>>> handling
> > > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> > > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this
> > makes
> > > > the
> > > > > >>>>>>> function
> > > > > >>>>>>>>> much
> > > > > >>>>>>>>>>>>>>>>>>> easier
> > > > > >>>>>>>>>>>>>>>>>>>           to use.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps
> > > resolve
> > > > > your
> > > > > >>>>>>>>> questions,
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>           please feel
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your
> > mind.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36
> > PM,
> > > > > >>>>> Matthias J.
> > > > > >>>>>>>>> Sax <
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io
> > <mailto:
> > > > > >>>>>>>>>>>> matthias@confluent.io>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on
> > this
> > > > > >> thread. I
> > > > > >>>>>>> did
> > > > > >>>>>>>>> not
> > > > > >>>>>>>>>>>> read
> > > > > >>>>>>>>>>>>>>>>>>>           everything so
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple
> of
> > > > > initial
> > > > > >>>>>>> thoughts:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is
> a
> > > > > >> fundamental
> > > > > >>>>>>>>>>>> difference
> > > > > >>>>>>>>>>>>>>>>>>>           between header
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258.
> For
> > > 258,
> > > > > we
> > > > > >> add
> > > > > >>>>>>>>> headers
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams
> > and
> > > > > nobody
> > > > > >>>>>>> else is
> > > > > >>>>>>>>>>>> supposed
> > > > > >>>>>>>>>>>>>>>>>>>           to write
> > > > > >>>>>>>>>>>>>>>>>>>           > into
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user
> > header
> > > > are
> > > > > >>>>> written
> > > > > >>>>>>> into
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
> > conflicts.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see
> a
> > > big
> > > > > issue
> > > > > >>>>> with
> > > > > >>>>>>>>> using
> > > > > >>>>>>>>>>>>>>>>>>>           headers within
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it,
> > we
> > > > can
> > > > > >> have
> > > > > >>>>>>> some
> > > > > >>>>>>>>>>>> "reserved"
> > > > > >>>>>>>>>>>>>>>>>>>           header keys
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to
> use
> > > > when
> > > > > >>>>>>> processing
> > > > > >>>>>>>>>>>> data with
> > > > > >>>>>>>>>>>>>>>>>>>           Kafka
> > > > > >>>>>>>>>>>>>>>>>>>           > Streams.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe
> way
> > to
> > > > > avoid
> > > > > >>>>>>>>> conflicts,
> > > > > >>>>>>>>>>>> since
> > > > > >>>>>>>>>>>>>>>>>>> these
> > > > > >>>>>>>>>>>>>>>>>>>           > headers
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal
> > > topics
> > > > (I
> > > > > >>>>> think):
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and
> changelog
> > > > > topics,
> > > > > >> we
> > > > > >>>>> can
> > > > > >>>>>>>>>>>> namespace
> > > > > >>>>>>>>>>>>>>>>>>>           all headers:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers
> are
> > > > > >> namespaced
> > > > > >>>>> as
> > > > > >>>>>>>>>>>> "external."
> > > > > >>>>>>>>>>>>>>>>>>> +
> > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> > > > > namespaced as
> > > > > >>>>>>>>>>>> "internal." +
> > > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would
> be
> > > > > >> possible,
> > > > > >>>>> it
> > > > > >>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>> require
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies
> a
> > > > > runtime
> > > > > >>>>>>> overhead.
> > > > > >>>>>>>>> I
> > > > > >>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid
> > the
> > > > > >>>>> overhead.
> > > > > >>>>>>> If
> > > > > >>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still
> add
> > > > name
> > > > > >>>>> spacing
> > > > > >>>>>>>>> later
> > > > > >>>>>>>>>>>> on.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the
> > > design
> > > > > it
> > > > > >> the
> > > > > >>>>>>> type
> > > > > >>>>>>>>> of
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
> > > > correctly,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable
> =
> > > > > >>>>>>>>>>>> table1.join(table2,...);
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
> > `joinedTable`
> > > > has
> > > > > >> the
> > > > > >>>>>>> same
> > > > > >>>>>>>>> key
> > > > > >>>>>>>>>>>> as the
> > > > > >>>>>>>>>>>>>>>>>>>           left input
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work
> > > because
> > > > > if
> > > > > >>>>> table2
> > > > > >>>>>>>>>>>> contains
> > > > > >>>>>>>>>>>>>>>>>>>           multiple rows
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in
> table1
> > > > > (what is
> > > > > >>>>> the
> > > > > >>>>>>> main
> > > > > >>>>>>>>>>>> purpose
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>           > foreign
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table
> > would
> > > > only
> > > > > >>>>>>> contain a
> > > > > >>>>>>>>>>>> single
> > > > > >>>>>>>>>>>>>>>>>>>           join result,
> > > > > >>>>>>>>>>>>>>>>>>>           > but
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
> > > <a,(A,1)>,
> > > > > >>>>> <b,(A,2)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a
> > foreign
> > > > key
> > > > > to
> > > > > >>>>>>> table1
> > > > > >>>>>>>>> key
> > > > > >>>>>>>>>>>> (ie,
> > > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
> > > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key
> > as
> > > > key
> > > > > of
> > > > > >>>>>>> table1,
> > > > > >>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>           implies that the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> > > > > join(X,1)>
> > > > > >> or
> > > > > >>>>> <A,
> > > > > >>>>>>>>>>>> join(X,2)>
> > > > > >>>>>>>>>>>>>>>>>>>           but not
> > > > > >>>>>>>>>>>>>>>>>>>           > both.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same
> > > key,
> > > > > >>>>> whatever
> > > > > >>>>>>>>> result
> > > > > >>>>>>>>>>>> record
> > > > > >>>>>>>>>>>>>>>>>>>           we emit
> > > > > >>>>>>>>>>>>>>>>>>>           > later,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous
> > result.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan
> > > > > originally
> > > > > >>>>>>> proposed
> > > > > >>>>>>>>>>>> to use
> > > > > >>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>           > combination
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the
> > input
> > > > > tables
> > > > > >>>>> as
> > > > > >>>>>>> key
> > > > > >>>>>>>>> of
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           output table.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the
> output
> > > > table
> > > > > >>>>> unique
> > > > > >>>>>>> and
> > > > > >>>>>>>>> we
> > > > > >>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>           store both in
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> > > > join(X,1)>,
> > > > > >> <A-b,
> > > > > >>>>>>>>>>>> join(X,2)>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan
> > > Filipiak
> > > > > >> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could
> be
> > > > > >>>>> disregarded.
> > > > > >>>>>>> The
> > > > > >>>>>>>>>>>> decision
> > > > > >>>>>>>>>>>>>>>>>>>           about the
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of
> the
> > > > > >> aggregated
> > > > > >>>>>>> map.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps
> > > would
> > > > be
> > > > > >>>>>>> unpacked
> > > > > >>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> > > > > >>>>>>>>>>>>>>>>>>>           > element
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as
> > delete.
> > > > Any
> > > > > >>>>> other
> > > > > >>>>>>> count
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in
> > "waiting
> > > > for
> > > > > >>>>> correct
> > > > > >>>>>>>>>>>> deletes to
> > > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam
> > > > > Bellemare
> > > > > >>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could
> > > > replace
> > > > > >> the
> > > > > >>>>>>> second
> > > > > >>>>>>>>>>>>>>>>>>>           repartition store
> > > > > >>>>>>>>>>>>>>>>>>>           > and
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a
> > > groupBy
> > > > > and
> > > > > >>>>>>> reduce.
> > > > > >>>>>>>>>>>> However,
> > > > > >>>>>>>>>>>>>>>>>>>           it looks
> > > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the
> > > > > highwater
> > > > > >>>>> value
> > > > > >>>>>>>>> within
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           materialized
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> > > > > out-of-order
> > > > > >>>>>>> records
> > > > > >>>>>>>>>>>> (assuming
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> my
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This
> > in
> > > > > effect
> > > > > >> is
> > > > > >>>>>>> the
> > > > > >>>>>>>>> same
> > > > > >>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>           design I
> > > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables
> > > merged
> > > > > >>>>> together.
> > > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > >>>>>>>>>>>>>>>>>>>           >
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>>> -- Guozhang
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>> -- Guozhang
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
> -- Guozhang
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

Thanks a lot for the suggestions on refactoring the wiki, I agree with you
that we should consider the KIP proposal to be easily understood by anyone
in the future to read, and hence should provide a good summary on the
user-facing interfaces, as well as rejected alternatives to represent
briefly "how we came a long way to this conclusion, and what we have
argued, disagreed, and agreed about, etc" so that readers do not need to
dig into the DISCUSS thread to get all the details. We can, of course, keep
the implementation details like "workflows" on the wiki page as a addendum
section since it also has correlations.

Regarding your proposal on comment 6): that's a very interesting idea! Just
to clarify that I understands it fully correctly: the proposal's resulted
topology is still the same as the current proposal, where we will have 3
sub-topologies for this operator:

#1: rekey left table
   -> source from the left upstream, send to rekey-processor to generate
combined key, and then sink to copartition topic.

#2: first-join with right table
   -> source from the right table upstream, materialize the right table.
   -> source from the co-partition topic, materialize the rekeyed left
table, join with the right table, rekey back, and then sink to the
rekeyed-back topic.

#3: second join
   -> source from the rekeyed-back topic, materialize the rekeyed back
table.
   -> source from the left upstream, materialize the left table, join with
the rekeyed back table.

Sub-topology #1 and #3 may be merged to a single sub-topology since both of
them read from the left table source stream. In this workflow, we need to
materialize 4 tables (left table in #3, right table in #2, rekeyed left
table in #2, rekeyed-back table in #3), and 2 repartition topics
(copartition topic, rekeyed-back topic).

Compared with Adam's current proposal in the workflow overview, it has the
same num.materialize tables (left table, rekeyed left table, right table,
out-of-ordering resolver table), and same num.internal topics (two). The
advantage is that on the copartition topic, we can save bandwidth by not
sending value, and in #2 the rekeyed left table is smaller since we do not
have any values to materialize. Is that right?


Guozhang



On Wed, Dec 12, 2018 at 1:22 PM John Roesler <jo...@confluent.io> wrote:

> Hi Adam,
>
> Given that the committers are all pretty busy right now, I think that it
> would help if you were to refactor the KIP a little to reduce the workload
> for reviewers.
>
> I'd recommend the following changes:
> * relocate all internal details to a section at the end called something
> like "Implementation Notes" or something like that.
> * rewrite the rest of the KIP to be a succinct as possible and mention only
> publicly-facing API changes.
> ** for example, the interface that you've already listed there, as well as
> a textual description of the guarantees we'll be providing (join result is
> copartitioned with the LHS, and the join result is guaranteed correct)
>
> A good target would be that the whole main body of the KIP, including
> Status, Motivation, Proposal, Justification, and Rejected Alternatives all
> fit "above the fold" (i.e., all fit on the screen at a comfortable zoom
> level).
> I think the only real Rejected Alternative that bears mention at this point
> is KScatteredTable, which you could just include the executive summary on
> (no implementation details), and link to extra details in the
> Implementation Notes section.
>
> Taking a look at the wiki page, ~90% of the text there is internal detail,
> which is useful for the dubious, but doesn't need to be ratified in a vote
> (and would be subject to change without notice in the future anyway).
> There's also a lot of conflicting discussion, as you've very respectfully
> tried to preserve the original proposal from Jan while adding your own.
> Isolating all this information in a dedicated section at the bottom frees
> the voters up to focus on the public API part of the proposal, which is
> really all they need to consider.
>
> Plus, it'll be clear to future readers which parts of the document are
> enduring, and which parts are a snapshot of our implementation thinking at
> the time.
>
> I'm suggesting this because I suspect that the others haven't made time to
> review it partly because it seems daunting. If it seems like it would be a
> huge time investment to review, people will just keep putting it off. But
> if the KIP is a single page, then they'll be more inclined to give it a
> read.
>
> Honestly, I don't think the KIP itself is that controversial (apart from
> the scattered table thing (sorry, Jan) ). Most of the discussion has been
> around the implementation, which we can continue more effectively in a PR
> once the KIP has passed.
>
> How does that sound?
> -John
>
> On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > 1) I believe that the resolution mechanism John has proposed is
> sufficient
> > - it is clean and easy and doesn't require additional RocksDB stores,
> which
> > reduces the footprint greatly. I don't think we need to resolve based on
> > timestamp or offset anymore, but if we decide to do to that would be
> within
> > the bounds of the existing API.
> >
> > 2) Is the current API sufficient, or does it need to be altered to go
> back
> > to vote?
> >
> > 3) KScatteredTable implementation can always be added in a future
> revision.
> > This API does not rule it out. This implementation of this function would
> > simply be replaced with `KScatteredTable.resolve()` while still
> maintaining
> > the existing API, thereby giving both features as Jan outlined earlier.
> > Would this work?
> >
> >
> > Thanks Guozhang, John and Jan
> >
> >
> >
> >
> > On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io> wrote:
> >
> > > Hi, all,
> > >
> > > >> In fact, we
> > > >> can just keep a single final-result store with timestamps and reject
> > > values
> > > >> that have a smaller timestamp, is that right?
> > >
> > > > Which is the correct output should at least be decided on the offset
> of
> > > > the original message.
> > >
> > > Thanks for this point, Jan.
> > >
> > > KIP-258 is merely to allow embedding the record timestamp  in the k/v
> > > store,
> > > as well as providing a storage-format upgrade path.
> > >
> > > I might have missed it, but I think we have yet to discuss whether it's
> > > safe
> > > or desirable just to swap topic-ordering our for timestamp-ordering.
> This
> > > is
> > > a very deep topic, and I think it would only pollute the current
> > > discussion.
> > >
> > > What Adam has proposed is safe, given the *current* ordering semantics
> > > of the system. If we can agree on his proposal, I think we can merge
> the
> > > feature well before the conversation about timestamp ordering even
> takes
> > > place, much less reaches a conclusion. In the mean time, it would seem
> to
> > > be unfortunate to have one join operator with different ordering
> > semantics
> > > from every other KTable operator.
> > >
> > > If and when that timestamp discussion takes place, many (all?) KTable
> > > operations
> > > will need to be updated, rendering the many:one join a small marginal
> > cost.
> > >
> > > And, just to plug it again, I proposed an algorithm above that I
> believe
> > > provides
> > > correct ordering without any additional metadata, and regardless of the
> > > ordering semantics. I didn't bring it up further, because I felt the
> KIP
> > > only needs
> > > to agree on the public API, and we can discuss the implementation at
> > > leisure in
> > > a PR...
> > >
> > > Thanks,
> > > -John
> > >
> > >
> > > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <Jan.Filipiak@trivago.com
> >
> > > wrote:
> > >
> > > >
> > > >
> > > > On 10.12.2018 07:42, Guozhang Wang wrote:
> > > > > Hello Adam / Jan / John,
> > > > >
> > > > > Sorry for being late on this thread! I've finally got some time
> this
> > > > > weekend to cleanup a load of tasks on my queue (actually I've also
> > > > realized
> > > > > there are a bunch of other things I need to enqueue while cleaning
> > them
> > > > up
> > > > > --- sth I need to improve on my side). So here are my thoughts:
> > > > >
> > > > > Regarding the APIs: I like the current written API in the KIP. More
> > > > > generally I'd prefer to keep the 1) one-to-many join
> functionalities
> > as
> > > > > well as 2) other join types than inner as separate KIPs since 1)
> may
> > > > worth
> > > > > a general API refactoring that can benefit not only foreignkey
> joins
> > > but
> > > > > collocate joins as well (e.g. an extended proposal of
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > ),
> > > > > and I'm not sure if other join types would actually be needed
> (maybe
> > > left
> > > > > join still makes sense), so it's better to
> > > wait-for-people-to-ask-and-add
> > > > > than add-sth-that-no-one-uses.
> > > > >
> > > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > > > > KScatteredTable for users to inject their own optimization: I'd
> > prefer
> > > to
> > > > > do the current option as-is, and my main rationale is for
> > optimization
> > > > > rooms inside the Streams internals and the API succinctness. For
> > > advanced
> > > > > users who may indeed prefer KScatteredTable and do their own
> > > > optimization,
> > > > > while it is too much of the work to use Processor API directly, I
> > think
> > > > we
> > > > > can still extend the current API to support it in the future if it
> > > > becomes
> > > > > necessary.
> > > >
> > > > no internal optimization potential. it's a myth
> > > >
> > > > ¯\_(ツ)_/¯
> > > >
> > > > :-)
> > > >
> > > > >
> > > > > Another note about step 4) resolving out-of-ordering data, as I
> > > mentioned
> > > > > before I think with KIP-258 (embedded timestamp with key-value
> store)
> > > we
> > > > > can actually make this step simpler than the current proposal. In
> > fact,
> > > > we
> > > > > can just keep a single final-result store with timestamps and
> reject
> > > > values
> > > > > that have a smaller timestamp, is that right?
> > > >
> > > > Which is the correct output should at least be decided on the offset
> of
> > > > the original message.
> > > >
> > > > >
> > > > >
> > > > > That's all I have in mind now. Again, great appreciation to Adam to
> > > make
> > > > > such HUGE progress on this KIP!
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> > Jan.Filipiak@trivago.com>
> > > > > wrote:
> > > > >
> > > > >> If they don't find the time:
> > > > >> They usually take the opposite path from me :D
> > > > >> so the answer would be clear.
> > > > >>
> > > > >> hence my suggestion to vote.
> > > > >>
> > > > >>
> > > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> > > > >>> Hi Guozhang and Matthias
> > > > >>>
> > > > >>> I know both of you are quite busy, but we've gotten this KIP to a
> > > point
> > > > >>> where we need more guidance on the API (perhaps a bit of a
> > > tie-breaker,
> > > > >> if
> > > > >>> you will). If you have anyone else you may think should look at
> > this,
> > > > >>> please tag them accordingly.
> > > > >>>
> > > > >>> The scenario is as such:
> > > > >>>
> > > > >>> Current Option:
> > > > >>> API:
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the
> partition
> > > with
> > > > >> the
> > > > >>> foreignKey (repartition 1)
> > > > >>> 2) Join the data
> > > > >>> 3) Shuffle the data back to the original node (repartition 2)
> > > > >>> 4) Resolve out-of-order arrival / race condition due to
> foreign-key
> > > > >> changes.
> > > > >>>
> > > > >>> Alternate Option:
> > > > >>> Perform #1 and #2 above, and return a KScatteredTable.
> > > > >>> - It would be keyed on a wrapped key function: <CombinedKey<KO,
> K>,
> > > VR>
> > > > >> (KO
> > > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> > > > >>> - KScatteredTable.resolve() would perform #3 and #4 but
> otherwise a
> > > > user
> > > > >>> would be able to perform additional functions directly from the
> > > > >>> KScatteredTable (TBD - currently out of scope).
> > > > >>> - John's analysis 2-emails up is accurate as to the tradeoffs.
> > > > >>>
> > > > >>> Current Option is coded as-is. Alternate option is possible, but
> > will
> > > > >>> require for implementation details to be made in the API and some
> > > > >> exposure
> > > > >>> of new data structures into the API (ie: CombinedKey).
> > > > >>>
> > > > >>> I appreciate any insight into this.
> > > > >>>
> > > > >>> Thanks.
> > > > >>>
> > > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > > > adam.bellemare@gmail.com>
> > > > >>> wrote:
> > > > >>>
> > > > >>>> Hi John
> > > > >>>>
> > > > >>>> Thanks for your feedback and assistance. I think your summary is
> > > > >> accurate
> > > > >>>> from my perspective. Additionally, I would like to add that
> there
> > > is a
> > > > >> risk
> > > > >>>> of inconsistent final states without performing the resolution.
> > This
> > > > is
> > > > >> a
> > > > >>>> major concern for me as most of the data I have dealt with is
> > > produced
> > > > >> by
> > > > >>>> relational databases. We have seen a number of cases where a
> user
> > in
> > > > the
> > > > >>>> Rails UI has modified the field (foreign key), realized they
> made
> > a
> > > > >>>> mistake, and then updated the field again with a new key. The
> > events
> > > > are
> > > > >>>> propagated out as they are produced, and as such we have had
> > > > real-world
> > > > >>>> cases where these inconsistencies were propagated downstream as
> > the
> > > > >> final
> > > > >>>> values due to the race conditions in the fanout of the data.
> > > > >>>>
> > > > >>>> This solution that I propose values correctness of the final
> > result
> > > > over
> > > > >>>> other factors.
> > > > >>>>
> > > > >>>> We could always move this function over to using a
> KScatteredTable
> > > > >>>> implementation in the future, and simply deprecate it this join
> > API
> > > in
> > > > >>>> time. I think I would like to hear more from some of the other
> > major
> > > > >>>> committers on which course of action they would think is best
> > before
> > > > any
> > > > >>>> more coding is done.
> > > > >>>>
> > > > >>>> Thanks again
> > > > >>>>
> > > > >>>> Adam
> > > > >>>>
> > > > >>>>
> > > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io>
> > > > wrote:
> > > > >>>>
> > > > >>>>> Hi Jan and Adam,
> > > > >>>>>
> > > > >>>>> Wow, thanks for doing that test, Adam. Those results are
> > > encouraging.
> > > > >>>>>
> > > > >>>>> Thanks for your performance experience as well, Jan. I agree
> that
> > > > >> avoiding
> > > > >>>>> unnecessary join outputs is especially important when the
> fan-out
> > > is
> > > > so
> > > > >>>>> high. I suppose this could also be built into the
> implementation
> > > > we're
> > > > >>>>> discussing, but it wouldn't have to be specified in the KIP
> > (since
> > > > >> it's an
> > > > >>>>> API-transparent optimization).
> > > > >>>>>
> > > > >>>>> As far as whether or not to re-repartition the data, I didn't
> > bring
> > > > it
> > > > >> up
> > > > >>>>> because it sounded like the two of you agreed to leave the KIP
> > > as-is,
> > > > >>>>> despite the disagreement.
> > > > >>>>>
> > > > >>>>> If you want my opinion, I feel like both approaches are
> > reasonable.
> > > > >>>>> It sounds like Jan values more the potential for developers to
> > > > optimize
> > > > >>>>> their topologies to re-use the intermediate nodes, whereas Adam
> > > > places
> > > > >>>>> more
> > > > >>>>> value on having a single operator that people can use without
> > extra
> > > > >> steps
> > > > >>>>> at the end.
> > > > >>>>>
> > > > >>>>> Personally, although I do find it exceptionally annoying when a
> > > > >> framework
> > > > >>>>> gets in my way when I'm trying to optimize something, it seems
> > > better
> > > > >> to
> > > > >>>>> go
> > > > >>>>> for a single operation.
> > > > >>>>> * Encapsulating the internal transitions gives us significant
> > > > latitude
> > > > >> in
> > > > >>>>> the implementation (for example, joining only at the end, not
> in
> > > the
> > > > >>>>> middle
> > > > >>>>> to avoid extra data copying and out-of-order resolution; how we
> > > > >> represent
> > > > >>>>> the first repartition keys (combined keys vs. value vectors),
> > > etc.).
> > > > >> If we
> > > > >>>>> publish something like a KScatteredTable with the
> > right-partitioned
> > > > >> joined
> > > > >>>>> data, then the API pretty much locks in the implementation as
> > well.
> > > > >>>>> * The API seems simpler to understand and use. I do mean
> "seems";
> > > if
> > > > >>>>> anyone
> > > > >>>>> wants to make the case that KScatteredTable is actually
> simpler,
> > I
> > > > >> think
> > > > >>>>> hypothetical usage code would help. From a relational algebra
> > > > >> perspective,
> > > > >>>>> it seems like KTable.join(KTable) should produce a new KTable
> in
> > > all
> > > > >>>>> cases.
> > > > >>>>> * That said, there might still be room in the API for a
> different
> > > > >>>>> operation
> > > > >>>>> like what Jan has proposed to scatter a KTable, and then do
> > things
> > > > like
> > > > >>>>> join, re-group, etc from there... I'm not sure; I haven't
> thought
> > > > >> through
> > > > >>>>> all the consequences yet.
> > > > >>>>>
> > > > >>>>> This is all just my opinion after thinking over the discussion
> so
> > > > >> far...
> > > > >>>>> -John
> > > > >>>>>
> > > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > > > >> adam.bellemare@gmail.com>
> > > > >>>>> wrote:
> > > > >>>>>
> > > > >>>>>> Updated the PR to take into account John's feedback.
> > > > >>>>>>
> > > > >>>>>> I did some preliminary testing for the performance of the
> > > > prefixScan.
> > > > >> I
> > > > >>>>>> have attached the file, but I will also include the text in
> the
> > > body
> > > > >>>>> here
> > > > >>>>>> for archival purposes (I am not sure what happens to attached
> > > > files).
> > > > >> I
> > > > >>>>>> also updated the PR and the KIP accordingly.
> > > > >>>>>>
> > > > >>>>>> Summary: It scales exceptionally well for scanning large
> values
> > of
> > > > >>>>>> records. As Jan mentioned previously, the real issue would be
> > more
> > > > >>>>> around
> > > > >>>>>> processing the resulting records after obtaining them. For
> > > instance,
> > > > >> it
> > > > >>>>>> takes approximately ~80-120 mS to flush the buffer and a
> further
> > > > >>>>> ~35-85mS
> > > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them.
> > > Iterating
> > > > >>>>>> through the records just to generate a simple count takes ~ 40
> > > times
> > > > >>>>> longer
> > > > >>>>>> than the flush + scan combined.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> ============================================================================================
> > > > >>>>>> Setup:
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> ============================================================================================
> > > > >>>>>> Java 9 with default settings aside from a 512 MB heap
> (Xmx512m,
> > > > >> Xms512m)
> > > > >>>>>> CPU: i7 2.2 Ghz.
> > > > >>>>>>
> > > > >>>>>> Note: I am using a slightly-modified, directly-accessible
> Kafka
> > > > >> Streams
> > > > >>>>>> RocksDB
> > > > >>>>>> implementation (RocksDB.java, basically just avoiding the
> > > > >>>>>> ProcessorContext).
> > > > >>>>>> There are no modifications to the default RocksDB values
> > provided
> > > in
> > > > >> the
> > > > >>>>>> 2.1/trunk release.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> keysize = 128 bytes
> > > > >>>>>> valsize = 512 bytes
> > > > >>>>>>
> > > > >>>>>> Step 1:
> > > > >>>>>> Write X positive matching events: (key = prefix + left-padded
> > > > >>>>>> auto-incrementing integer)
> > > > >>>>>> Step 2:
> > > > >>>>>> Write 10X negative matching events (key = left-padded
> > > > >> auto-incrementing
> > > > >>>>>> integer)
> > > > >>>>>> Step 3:
> > > > >>>>>> Perform flush
> > > > >>>>>> Step 4:
> > > > >>>>>> Perform prefixScan
> > > > >>>>>> Step 5:
> > > > >>>>>> Iterate through return Iterator and validate the count of
> > expected
> > > > >>>>> events.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> ============================================================================================
> > > > >>>>>> Results:
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> ============================================================================================
> > > > >>>>>> X = 1k (11k events total)
> > > > >>>>>> Flush Time = 39 mS
> > > > >>>>>> Scan Time = 7 mS
> > > > >>>>>> 6.9 MB disk
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > >>>>>> X = 10k (110k events total)
> > > > >>>>>> Flush Time = 45 mS
> > > > >>>>>> Scan Time = 8 mS
> > > > >>>>>> 127 MB
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > >>>>>> X = 100k (1.1M events total)
> > > > >>>>>> Test1:
> > > > >>>>>> Flush Time = 60 mS
> > > > >>>>>> Scan Time = 12 mS
> > > > >>>>>> 678 MB
> > > > >>>>>>
> > > > >>>>>> Test2:
> > > > >>>>>> Flush Time = 45 mS
> > > > >>>>>> Scan Time = 7 mS
> > > > >>>>>> 576 MB
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > >>>>>> X = 1MB (11M events total)
> > > > >>>>>> Test1:
> > > > >>>>>> Flush Time = 52 mS
> > > > >>>>>> Scan Time = 19 mS
> > > > >>>>>> 7.2 GB
> > > > >>>>>>
> > > > >>>>>> Test2:
> > > > >>>>>> Flush Time = 84 mS
> > > > >>>>>> Scan Time = 34 mS
> > > > >>>>>> 9.1 GB
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > >>>>>> X = 2.5M (27.5M events total)
> > > > >>>>>> Test1:
> > > > >>>>>> Flush Time = 82 mS
> > > > >>>>>> Scan Time = 63 mS
> > > > >>>>>> 17GB - 276 sst files
> > > > >>>>>>
> > > > >>>>>> Test2:
> > > > >>>>>> Flush Time = 116 mS
> > > > >>>>>> Scan Time = 35 mS
> > > > >>>>>> 23GB - 361 sst files
> > > > >>>>>>
> > > > >>>>>> Test3:
> > > > >>>>>> Flush Time = 103 mS
> > > > >>>>>> Scan Time = 82 mS
> > > > >>>>>> 19 GB - 300 sst files
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> --------------------------------------------------------------------------------------------
> > > > >>>>>>
> > > > >>>>>> I had to limit my testing on my laptop to X = 2.5M events. I
> > tried
> > > > to
> > > > >> go
> > > > >>>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+
> > > range
> > > > >>>>> and my
> > > > >>>>>> laptop ran out of disk. More extensive testing could be done
> > but I
> > > > >>>>> suspect
> > > > >>>>>> that it would be in line with what we're seeing in the results
> > > > above.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> At this point in time, I think the only major discussion point
> > is
> > > > >> really
> > > > >>>>>> around what Jan and I have disagreed on: repartitioning back +
> > > > >> resolving
> > > > >>>>>> potential out of order issues or leaving that up to the client
> > to
> > > > >>>>> handle.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> Thanks folks,
> > > > >>>>>>
> > > > >>>>>> Adam
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > > > Jan.Filipiak@trivago.com
> > > > >>>
> > > > >>>>>> wrote:
> > > > >>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > > > >>>>>>>> Hi all,
> > > > >>>>>>>>
> > > > >>>>>>>> Sorry that this discussion petered out... I think the 2.1
> > > release
> > > > >>>>>>> caused an
> > > > >>>>>>>> extended distraction that pushed it off everyone's radar
> > (which
> > > > was
> > > > >>>>>>>> precisely Adam's concern). Personally, I've also had some
> > extend
> > > > >>>>>>>> distractions of my own that kept (and continue to keep) me
> > > > >>>>> preoccupied.
> > > > >>>>>>>>
> > > > >>>>>>>> However, calling for a vote did wake me up, so I guess Jan
> was
> > > on
> > > > >> the
> > > > >>>>>>> right
> > > > >>>>>>>> track!
> > > > >>>>>>>>
> > > > >>>>>>>> I've gone back and reviewed the whole KIP document and the
> > prior
> > > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> > > > >>>>>>>>
> > > > >>>>>>>> API Thoughts:
> > > > >>>>>>>>
> > > > >>>>>>>> 1. If I read the KIP right, you are proposing a many-to-one
> > > join.
> > > > >>>>> Could
> > > > >>>>>>> we
> > > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip
> the
> > > > design
> > > > >>>>>>> around
> > > > >>>>>>>> and make it a oneToManyJoin?
> > > > >>>>>>>>
> > > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join
> type,
> > > and
> > > > it
> > > > >>>>>>> seems
> > > > >>>>>>>> like it might trick some people into using it for a
> one-to-one
> > > > join.
> > > > >>>>>>> This
> > > > >>>>>>>> would work, of course, but it would be super inefficient
> > > compared
> > > > to
> > > > >>>>> a
> > > > >>>>>>>> simple rekey-and-join.
> > > > >>>>>>>>
> > > > >>>>>>>> 2. I might have missed it, but I don't think it's specified
> > > > whether
> > > > >>>>>>> it's an
> > > > >>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
> > > > >>>>> (neglecting
> > > > >>>>>>> IQ),
> > > > >>>>>>>> the rest can be achieved by filtering or by handling it in
> the
> > > > >>>>>>> ValueJoiner.
> > > > >>>>>>>>
> > > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite
> right.
> > > > >>>>>>>> 3a. Regarding Serialized: There are a few different
> paradigms
> > in
> > > > >>>>> play in
> > > > >>>>>>>> the Streams API, so it's confusing, but instead of three
> > > > Serialized
> > > > >>>>>>> args, I
> > > > >>>>>>>> think it would be better to have one that allows
> (optionally)
> > > > >> setting
> > > > >>>>>>> the 4
> > > > >>>>>>>> incoming serdes. The result serde is defined by the
> > > Materialized.
> > > > >> The
> > > > >>>>>>>> incoming serdes can be optional because they might already
> be
> > > > >>>>> available
> > > > >>>>>>> on
> > > > >>>>>>>> the source KTables, or the default serdes from the config
> > might
> > > be
> > > > >>>>>>>> applicable.
> > > > >>>>>>>>
> > > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins
> don't
> > > > allow
> > > > >>>>>>> setting
> > > > >>>>>>>> one, and it seems like it might actually be harmful, since
> the
> > > > rekey
> > > > >>>>>>>> operation needs to produce results that are co-partitioned
> > with
> > > > the
> > > > >>>>>>> "other"
> > > > >>>>>>>> KTable.
> > > > >>>>>>>>
> > > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't
> > > actually
> > > > >>>>>>> follow
> > > > >>>>>>>> what Matthias meant about namespacing requiring
> > "deserializing"
> > > > the
> > > > >>>>>>> record
> > > > >>>>>>>> header. The headers are already Strings, so I don't think
> that
> > > > >>>>>>>> deserialization is required. If we applied the namespace at
> > > source
> > > > >>>>> nodes
> > > > >>>>>>>> and stripped it at sink nodes, this would be practically no
> > > > >> overhead.
> > > > >>>>>>> The
> > > > >>>>>>>> advantage of the namespace idea is that no public API change
> > wrt
> > > > >>>>> headers
> > > > >>>>>>>> needs to happen, and no restrictions need to be placed on
> > users'
> > > > >>>>>>> headers.
> > > > >>>>>>>>
> > > > >>>>>>>> (Although I'm wondering if we can get away without the
> header
> > at
> > > > >>>>> all...
> > > > >>>>>>>> stay tuned)
> > > > >>>>>>>>
> > > > >>>>>>>> 5. I also didn't follow the discussion about the HWM table
> > > growing
> > > > >>>>>>> without
> > > > >>>>>>>> bound. As I read it, the HWM table is effectively
> implementing
> > > OCC
> > > > >> to
> > > > >>>>>>>> resolve the problem you noted with disordering when the
> rekey
> > is
> > > > >>>>>>>> reversed... particularly notable when the FK changes. As
> such,
> > > it
> > > > >>>>> only
> > > > >>>>>>>> needs to track the most recent "version" (the offset in the
> > > source
> > > > >>>>>>>> partition) of each key. Therefore, it should have the same
> > > number
> > > > of
> > > > >>>>>>> keys
> > > > >>>>>>>> as the source table at all times.
> > > > >>>>>>>>
> > > > >>>>>>>> I see that you are aware of KIP-258, which I think might be
> > > > relevant
> > > > >>>>> in
> > > > >>>>>>> a
> > > > >>>>>>>> couple of ways. One: it's just about storing the timestamp
> in
> > > the
> > > > >>>>> state
> > > > >>>>>>>> store, but the ultimate idea is to effectively use the
> > timestamp
> > > > as
> > > > >>>>> an
> > > > >>>>>>> OCC
> > > > >>>>>>>> "version" to drop disordered updates. You wouldn't want to
> use
> > > the
> > > > >>>>>>>> timestamp for this operation, but if you were to use a
> similar
> > > > >>>>>>> mechanism to
> > > > >>>>>>>> store the source offset in the store alongside the re-keyed
> > > > values,
> > > > >>>>> then
> > > > >>>>>>>> you could avoid a separate table.
> > > > >>>>>>>>
> > > > >>>>>>>> 6. You and Jan have been thinking about this for a long
> time,
> > so
> > > > >> I've
> > > > >>>>>>>> probably missed something here, but I'm wondering if we can
> > > avoid
> > > > >> the
> > > > >>>>>>> HWM
> > > > >>>>>>>> tracking at all and resolve out-of-order during a final join
> > > > >>>>> instead...
> > > > >>>>>>>>
> > > > >>>>>>>> Let's say we're joining a left table (Integer K: Letter FK,
> > > (other
> > > > >>>>>>> data))
> > > > >>>>>>>> to a right table (Letter K: (some data)).
> > > > >>>>>>>>
> > > > >>>>>>>> Left table:
> > > > >>>>>>>> 1: (A, xyz)
> > > > >>>>>>>> 2: (B, asd)
> > > > >>>>>>>>
> > > > >>>>>>>> Right table:
> > > > >>>>>>>> A: EntityA
> > > > >>>>>>>> B: EntityB
> > > > >>>>>>>>
> > > > >>>>>>>> We could do a rekey as you proposed with a combined key, but
> > not
> > > > >>>>>>>> propagating the value at all..
> > > > >>>>>>>> Rekey table:
> > > > >>>>>>>> A-1: (dummy value)
> > > > >>>>>>>> B-2: (dummy value)
> > > > >>>>>>>>
> > > > >>>>>>>> Which we then join with the right table to produce:
> > > > >>>>>>>> A-1: EntityA
> > > > >>>>>>>> B-2: EntityB
> > > > >>>>>>>>
> > > > >>>>>>>> Which gets rekeyed back:
> > > > >>>>>>>> 1: A, EntityA
> > > > >>>>>>>> 2: B, EntityB
> > > > >>>>>>>>
> > > > >>>>>>>> And finally we do the actual join:
> > > > >>>>>>>> Result table:
> > > > >>>>>>>> 1: ((A, xyz), EntityA)
> > > > >>>>>>>> 2: ((B, asd), EntityB)
> > > > >>>>>>>>
> > > > >>>>>>>> The thing is that in that last join, we have the opportunity
> > to
> > > > >>>>> compare
> > > > >>>>>>> the
> > > > >>>>>>>> current FK in the left table with the incoming PK of the
> right
> > > > >>>>> table. If
> > > > >>>>>>>> they don't match, we just drop the event, since it must be
> > > > outdated.
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets
> > > updated
> > > > >> to
> > > > >>>>>>> (1:
> > > > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether the
> > final
> > > > >>>>> state
> > > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm
> > > above,
> > > > >>>>> you
> > > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B,
> xyz),
> > > (B,
> > > > >>>>>>>> EntityB)). It seems like this does give you enough
> information
> > > to
> > > > >>>>> make
> > > > >>>>>>> the
> > > > >>>>>>>> right choice, regardless of disordering.
> > > > >>>>>>>
> > > > >>>>>>> Will check Adams patch, but this should work. As mentioned
> > often
> > > I
> > > > am
> > > > >>>>>>> not convinced on partitioning back for the user
> automatically.
> > I
> > > > >> think
> > > > >>>>>>> this is the real performance eater ;)
> > > > >>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> 7. Last thought... I'm a little concerned about the
> > performance
> > > of
> > > > >>>>> the
> > > > >>>>>>>> range scans when records change in the right table. You've
> > said
> > > > that
> > > > >>>>>>> you've
> > > > >>>>>>>> been using the algorithm you presented in production for a
> > > while.
> > > > >> Can
> > > > >>>>>>> you
> > > > >>>>>>>> give us a sense of the performance characteristics you've
> > > > observed?
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> Make it work, make it fast, make it beautiful. The topmost
> > thing
> > > > here
> > > > >>>>> is
> > > > >>>>>>> / was correctness. In practice I do not measure the
> performance
> > > of
> > > > >> the
> > > > >>>>>>> range scan. Usual cases I run this with is emitting 500k -
> 1kk
> > > rows
> > > > >>>>>>> on a left hand side change. The range scan is just the work
> you
> > > > gotta
> > > > >>>>>>> do, also when you pack your data into different formats,
> > usually
> > > > the
> > > > >>>>>>> rocks performance is very tight to the size of the data and
> we
> > > > can't
> > > > >>>>>>> really change that. It is more important for users to prevent
> > > > useless
> > > > >>>>>>> updates to begin with. My left hand side is guarded to drop
> > > changes
> > > > >>>>> that
> > > > >>>>>>> are not going to change my join output.
> > > > >>>>>>>
> > > > >>>>>>> usually it's:
> > > > >>>>>>>
> > > > >>>>>>> drop unused fields and then don't forward if old.equals(new)
> > > > >>>>>>>
> > > > >>>>>>> regarding to the performance of creating an iterator for
> > smaller
> > > > >>>>>>> fanouts, users can still just do a group by first then
> anyways.
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>> I could only think of one alternative, but I'm not sure if
> > it's
> > > > >>>>> better
> > > > >>>>>>> or
> > > > >>>>>>>> worse... If the first re-key only needs to preserve the
> > original
> > > > >> key,
> > > > >>>>>>> as I
> > > > >>>>>>>> proposed in #6, then we could store a vector of keys in the
> > > value:
> > > > >>>>>>>>
> > > > >>>>>>>> Left table:
> > > > >>>>>>>> 1: A,...
> > > > >>>>>>>> 2: B,...
> > > > >>>>>>>> 3: A,...
> > > > >>>>>>>>
> > > > >>>>>>>> Gets re-keyed:
> > > > >>>>>>>> A: [1, 3]
> > > > >>>>>>>> B: [2]
> > > > >>>>>>>>
> > > > >>>>>>>> Then, the rhs part of the join would only need a regular
> > > > single-key
> > > > >>>>>>> lookup.
> > > > >>>>>>>> Of course we have to deal with the problem of large values,
> as
> > > > >>>>> there's
> > > > >>>>>>> no
> > > > >>>>>>>> bound on the number of lhs records that can reference rhs
> > > records.
> > > > >>>>>>> Offhand,
> > > > >>>>>>>> I'd say we could page the values, so when one row is past
> the
> > > > >>>>>>> threshold, we
> > > > >>>>>>>> append the key for the next page. Then in most cases, it
> would
> > > be
> > > > a
> > > > >>>>>>> single
> > > > >>>>>>>> key lookup, but for large fan-out updates, it would be one
> per
> > > > (max
> > > > >>>>>>> value
> > > > >>>>>>>> size)/(avg lhs key size).
> > > > >>>>>>>>
> > > > >>>>>>>> This seems more complex, though... Plus, I think there's
> some
> > > > extra
> > > > >>>>>>>> tracking we'd need to do to know when to emit a retraction.
> > For
> > > > >>>>> example,
> > > > >>>>>>>> when record 1 is deleted, the re-key table would just have
> (A:
> > > > [3]).
> > > > >>>>>>> Some
> > > > >>>>>>>> kind of tombstone is needed so that the join result for 1
> can
> > > also
> > > > >> be
> > > > >>>>>>>> retracted.
> > > > >>>>>>>>
> > > > >>>>>>>> That's all!
> > > > >>>>>>>>
> > > > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP.
> > > Sorry
> > > > >> the
> > > > >>>>>>>> discussion has been slow.
> > > > >>>>>>>> -John
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > > > >>>>> Jan.Filipiak@trivago.com>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>
> > > > >>>>>>>>> Id say you can just call the vote.
> > > > >>>>>>>>>
> > > > >>>>>>>>> that happens all the time, and if something comes up, it
> just
> > > > goes
> > > > >>>>> back
> > > > >>>>>>>>> to discuss.
> > > > >>>>>>>>>
> > > > >>>>>>>>> would not expect to much attention with another another
> email
> > > in
> > > > >>>>> this
> > > > >>>>>>>>> thread.
> > > > >>>>>>>>>
> > > > >>>>>>>>> best Jan
> > > > >>>>>>>>>
> > > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > > > >>>>>>>>>> Hello Contributors
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> I know that 2.1 is about to be released, but I do need to
> > bump
> > > > >>>>> this to
> > > > >>>>>>>>> keep
> > > > >>>>>>>>>> visibility up. I am still intending to push this through
> > once
> > > > >>>>>>> contributor
> > > > >>>>>>>>>> feedback is given.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Main points that need addressing:
> > > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current
> singular
> > > > graph
> > > > >>>>> node
> > > > >>>>>>>>> into
> > > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right
> now. I
> > > am
> > > > a
> > > > >>>>> bit
> > > > >>>>>>>>> fuzzy
> > > > >>>>>>>>>> on how the optimizations are supposed to work, so I would
> > > > >>>>> appreciate
> > > > >>>>>>> any
> > > > >>>>>>>>>> help on this aspect.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This thread
> has
> > > > much
> > > > >>>>>>>>> discourse
> > > > >>>>>>>>>> between Jan and I between the current highwater mark
> > proposal
> > > > and
> > > > >> a
> > > > >>>>>>>>> groupBy
> > > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need to
> > > strictly
> > > > >>>>> handle
> > > > >>>>>>>>> any
> > > > >>>>>>>>>> chance of out-of-order data and leave none of it up to the
> > > > >>>>> consumer.
> > > > >>>>>>> Any
> > > > >>>>>>>>>> comments or suggestions here would also help.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 3) Anything else that you see that would prevent this from
> > > > moving
> > > > >>>>> to a
> > > > >>>>>>>>> vote?
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Thanks
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Adam
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > > > >>>>>>>>> adam.bellemare@gmail.com>
> > > > >>>>>>>>>> wrote:
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>> Hi Jan
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> > > > >>>>> Stores.persistentWindowStore,
> > > > >>>>>>> you
> > > > >>>>>>>>>>> actually only need to specify the amount of segments you
> > want
> > > > and
> > > > >>>>> how
> > > > >>>>>>>>> large
> > > > >>>>>>>>>>> they are. To the best of my understanding, what happens
> is
> > > that
> > > > >>>>> the
> > > > >>>>>>>>>>> segments are automatically rolled over as new data with
> new
> > > > >>>>>>> timestamps
> > > > >>>>>>>>> are
> > > > >>>>>>>>>>> created. We use this exact functionality in some of the
> > work
> > > > done
> > > > >>>>>>>>>>> internally at my company. For reference, this is the
> > hopping
> > > > >>>>> windowed
> > > > >>>>>>>>> store.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> In the code that I have provided, there are going to be
> two
> > > 24h
> > > > >>>>>>>>> segments.
> > > > >>>>>>>>>>> When a record is put into the windowStore, it will be
> > > inserted
> > > > at
> > > > >>>>>>> time
> > > > >>>>>>>>> T in
> > > > >>>>>>>>>>> both segments. The two segments will always overlap by
> 12h.
> > > As
> > > > >>>>> time
> > > > >>>>>>>>> goes on
> > > > >>>>>>>>>>> and new records are added (say at time T+12h+), the
> oldest
> > > > >> segment
> > > > >>>>>>> will
> > > > >>>>>>>>> be
> > > > >>>>>>>>>>> automatically deleted and a new segment created. The
> > records
> > > > are
> > > > >>>>> by
> > > > >>>>>>>>> default
> > > > >>>>>>>>>>> inserted with the context.timestamp(), such that it is
> the
> > > > record
> > > > >>>>>>> time,
> > > > >>>>>>>>> not
> > > > >>>>>>>>>>> the clock time, which is used.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> To the best of my understanding, the timestamps are
> > retained
> > > > when
> > > > >>>>>>>>>>> restoring from the changelog.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
> > > > >>>>>>> segment-level,
> > > > >>>>>>>>>>> instead of at an individual record level.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > > > >>>>>>> Jan.Filipiak@trivago.com>
> > > > >>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> Will that work? I expected it to blow up with
> > > > ClassCastException
> > > > >>>>> or
> > > > >>>>>>>>>>>> similar.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> You either would have to specify the window you
> fetch/put
> > or
> > > > >>>>> iterate
> > > > >>>>>>>>>>>> across all windows the key was found in right?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> I just hope the window-store doesn't check stream-time
> > under
> > > > the
> > > > >>>>>>> hoods
> > > > >>>>>>>>>>>> that would be a questionable interface.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> If it does: did you see my comment on checking all the
> > > windows
> > > > >>>>>>> earlier?
> > > > >>>>>>>>>>>> that would be needed to actually give reasonable time
> > > > gurantees.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Best
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > > > >>>>>>>>>>>>> Hi Jan
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed
> the
> > > > state
> > > > >>>>>>> store,
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>> the ProcessorSupplier.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>> Adam
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> @Guozhang
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed something
> > that
> > > > >>>>> will be
> > > > >>>>>>>>>>>>>>> extremely
> > > > >>>>>>>>>>>>>>> useful for this KIP.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> @Jan
> > > > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I will
> > not
> > > > be
> > > > >>>>>>> moving
> > > > >>>>>>>>>>>> ahead
> > > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy
> solution
> > > as
> > > > >> you
> > > > >>>>>>>>>>>> propose.
> > > > >>>>>>>>>>>>>>> That being said, if you wish to implement it yourself
> > off
> > > > of
> > > > >>>>> my
> > > > >>>>>>>>>>>> current PR
> > > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I would
> be
> > > more
> > > > >>>>> than
> > > > >>>>>>>>>>>> happy to
> > > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it stands
> > > right
> > > > >>>>> now,
> > > > >>>>>>> I do
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>> really have more time to invest into alternatives
> > without
> > > > >>>>> there
> > > > >>>>>>>>> being
> > > > >>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>> strong indication from the binding voters which they
> > > would
> > > > >>>>>>> prefer.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on
> > the
> > > > >>>>> streams
> > > > >>>>>>>>> DSL
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>> some time already, otherwise I would have pulled this
> > KIP
> > > > >>>>> through
> > > > >>>>>>>>>>>> already.
> > > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on
> PAPI.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed
> > state
> > > > >>>>> store
> > > > >>>>>>> in
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I will
> > come
> > > > >> back
> > > > >>>>>>> for
> > > > >>>>>>>>>>>> final
> > > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the
> > > > binding
> > > > >>>>>>> voters
> > > > >>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
> > > > >>>>> according
> > > > >>>>>>> to
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> latest plan:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed
> store.
> > > > This
> > > > >>>>>>> could
> > > > >>>>>>>>> be
> > > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
> > > > >>>>> completed.
> > > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Adam
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already
> > > > updated
> > > > >>>>> in
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> PR?
> > > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
> > > something?
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > > > >>>>>>> wangguoz@gmail.com>
> > > > >>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the
> > wrong
> > > > >> link,
> > > > >>>>>>> as it
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of
> > > KIP-258
> > > > >>>>> we do
> > > > >>>>>>>>>>>> want to
> > > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source KTable"
> > such
> > > > >> that
> > > > >>>>>>>>>>>> instead of
> > > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized store,
> > > i.e.
> > > > >>>>>>> following
> > > > >>>>>>>>>>>>>>>> offset
> > > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older than
> > the
> > > > >>>>> current
> > > > >>>>>>>>>>>> key's
> > > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Guozhang
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> > > > >>>>>>>>> wangguoz@gmail.com>
> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Hello Adam,
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final
> step
> > > > (i.e.
> > > > >>>>> the
> > > > >>>>>>>>> high
> > > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a
> > > window
> > > > >>>>>>> store),
> > > > >>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>> think
> > > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >>>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value
> > store
> > > > >>>>> (i.e.
> > > > >>>>>>> only
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
> > > > >>>>> described
> > > > >>>>>>> in
> > > > >>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533,
> is
> > > > that
> > > > >>>>> we
> > > > >>>>>>> can
> > > > >>>>>>>>>>>> then
> > > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
> > > timestamp
> > > > is
> > > > >>>>>>>>> smaller
> > > > >>>>>>>>>>>> than
> > > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I think
> it
> > > is
> > > > >>>>> very
> > > > >>>>>>>>>>>> similar to
> > > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
> > > filtering,
> > > > >>>>> while
> > > > >>>>>>>>> you
> > > > >>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the
> joining
> > > > >> records
> > > > >>>>>>> are
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> correctly
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the final
> > stage.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
> > > > >>>>>>> non-windowed
> > > > >>>>>>>>>>>> KTables
> > > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really
> have
> > a
> > > > good
> > > > >>>>>>>>> support
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>> their joins anyways (
> > > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > > > >>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed
> KTable-KTable
> > > > >>>>> non-key
> > > > >>>>>>>>> joins
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Guozhang
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Hi Guozhang
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
> > > > >> endlessly
> > > > >>>>>>> based
> > > > >>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of
> > (<this
> > > > >>>>> table
> > > > >>>>>>>>>>>> primary
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> key>,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used
> to
> > > > >>>>>>> differentiate
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> between
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal
> > would
> > > > be
> > > > >>>>> to
> > > > >>>>>>>>>>>> replace
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This
> would
> > > > allow
> > > > >>>>> the
> > > > >>>>>>>>> same
> > > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This
> > > should
> > > > >>>>> allow
> > > > >>>>>>> for
> > > > >>>>>>>>>>>> all
> > > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should
> be
> > > > >>>>>>> customizable
> > > > >>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps
> just
> > > 10
> > > > >>>>>>> minutes
> > > > >>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> window,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the
> > trick
> > > > >> here.
> > > > >>>>>>> Even
> > > > >>>>>>>>>>>> if I
> > > > >>>>>>>>>>>>>>>>>> would still like to see the automatic
> repartitioning
> > > > >>>>> optional
> > > > >>>>>>>>>>>> since I
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a
> > little
> > > > bit
> > > > >>>>>>>>>>>> sceptical
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one
> could
> > > run
> > > > >>>>> into
> > > > >>>>>>>>>>>> problems
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> the rapid change happens near a window border. I
> will
> > > > check
> > > > >>>>> you
> > > > >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we
> > could
> > > > >>>>> still
> > > > >>>>>>>>> check
> > > > >>>>>>>>>>>>>>>>>> _all_
> > > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance
> impact I
> > > > >> guess.
> > > > >>>>>>> Will
> > > > >>>>>>>>>>>> let
> > > > >>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct as
> is. I
> > > > >>>>> wouldn't
> > > > >>>>>>> not
> > > > >>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) =>
> timestamp(A)  <
> > > > >>>>>>>>> timestamp(B).
> > > > >>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> think
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> we can't expect that.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> @Jan
> > > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks
> > for
> > > > the
> > > > >>>>>>>>>>>> diagram, it
> > > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not
> have
> > > the
> > > > >>>>>>> original
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> primary
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was
> available
> > > > then
> > > > >>>>> you
> > > > >>>>>>>>>>>> would be
> > > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That
> > > being
> > > > >>>>> said,
> > > > >>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> encourage
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for
> clarity
> > > for
> > > > >>>>>>> everyone
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> else.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard
> > work.
> > > > But
> > > > >>>>> I
> > > > >>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the
> original
> > > > >> primary
> > > > >>>>>>> key,
> > > > >>>>>>>>> We
> > > > >>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
> > > > >> basically
> > > > >>>>>>> not
> > > > >>>>>>>>>>>> using
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that
> in
> > > > >>>>> original
> > > > >>>>>>> DSL
> > > > >>>>>>>>>>>> its
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on
> my
> > > end.
> > > > >>>>> Will
> > > > >>>>>>>>>>>> finish
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map
> stay
> > > > >> inside
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> State
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
> > > > >>>>> propagated?
> > > > >>>>>>>>> Isn't
> > > > >>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state
> > store?
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is
> > > gonna
> > > > >>>>>>> return
> > > > >>>>>>>>>>>> `null`
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is
> > > going
> > > > to
> > > > >>>>> be
> > > > >>>>>>> a
> > > > >>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store
> > > > directly
> > > > >>>>> for
> > > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a
> > regular
> > > > >>>>> store,
> > > > >>>>>>>>>>>> satisfying
> > > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join.
> The
> > > > >>>>> Windowed
> > > > >>>>>>>>>>>> store is
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull
> > operation
> > > > we
> > > > >>>>>>> would
> > > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the
> > > > window
> > > > >>>>>>> store
> > > > >>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> the values then.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group
> > by
> > > > >>>>> before
> > > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i think
> > it
> > > > >> would
> > > > >>>>>>> help
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> users
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the
> > original
> > > > >>>>> primary
> > > > >>>>>>>>> key
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> issue I
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> understand that we do not have a solid foundation
> to
> > > > build
> > > > >>>>> on.
> > > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
> > > > >>>>>>> unfortunate. I
> > > > >>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not
> > think
> > > > its
> > > > >>>>> a
> > > > >>>>>>> good
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> decision.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Thanks
> > > > >>>>>>>>>>>>>>>>>>> Adam
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta
> Dumbre <
> > > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > > > >>>>>>> dumbreprajakta311@gmail.com
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan
> > > Filipiak
> > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           wrote:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> > > > >>>>>>>>>>>>>>>>>>>           >
> > > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make such a
> > > chart.
> > > > >> last
> > > > >>>>>>> time i
> > > > >>>>>>>>>>>> didn't
> > > > >>>>>>>>>>>>>>>>>>>           get along
> > > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your
> > > charts.
> > > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> > > > >>>>>>>>>>>>>>>>>>>           >
> > > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare
> > > wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of what I
> > > > >> attempted
> > > > >>>>> on
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           >
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > > > >>>>>>>>>>>>>>>>>>>           <
> > > > >>>>>>>>>>>>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the start
> of
> > > my
> > > > own
> > > > >>>>>>>>>>>> implementation
> > > > >>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>           this
> > > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not get
> > it
> > > to
> > > > >>>>> work I
> > > > >>>>>>> have
> > > > >>>>>>>>>>>> since
> > > > >>>>>>>>>>>>>>>>>>>           discarded the
> > > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if you
> > > wish
> > > > to
> > > > >>>>>>> continue
> > > > >>>>>>>>>>>> pursuing
> > > > >>>>>>>>>>>>>>>>>>>           for your
> > > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you
> > please
> > > > >>>>> create a
> > > > >>>>>>>>>>>> diagram on
> > > > >>>>>>>>>>>>>>>>>>>           the KIP
> > > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your solution.
> > > > Please
> > > > >>>>> feel
> > > > >>>>>>> free
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>           the image I
> > > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting point. I
> am
> > > > having
> > > > >>>>>>> trouble
> > > > >>>>>>>>>>>>>>>>>>>           understanding your
> > > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a
> > > carefully
> > > > >>>>>>> constructed
> > > > >>>>>>>>>>>> diagram
> > > > >>>>>>>>>>>>>>>>>>>           will clear
> > > > >>>>>>>>>>>>>>>>>>>           > up
> > > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings. Alternately,
> > > > please
> > > > >>>>> post a
> > > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> > > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess at
> > what
> > > > you
> > > > >>>>>>> mean, and
> > > > >>>>>>>>>>>> since I
> > > > >>>>>>>>>>>>>>>>>>>           value my
> > > > >>>>>>>>>>>>>>>>>>>           > own
> > > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value yours, I
> > > > believe
> > > > >> it
> > > > >>>>> is
> > > > >>>>>>> your
> > > > >>>>>>>>>>>>>>>>>>>           responsibility to
> > > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation instead
> of
> > me
> > > > >>>>> trying to
> > > > >>>>>>>>> guess.
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > > Adam
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM, Jan
> > > > Filipiak
> > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > > wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >
> > > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing interested.
> > > kafka
> > > > >>>>>>> streams at
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>           point supports
> > > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as both
> > > > streams
> > > > >>>>> have
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> same
> > > > >>>>>>>>>>>>>>>>>>> key.
> > > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a
> > join
> > > > >> where a
> > > > >>>>>>> KTable
> > > > >>>>>>>>>>>> and a
> > > > >>>>>>>>>>>>>>>>>>>           KTable can
> > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship (1:n).
> > We
> > > > >> exploit
> > > > >>>>>>> that
> > > > >>>>>>>>>>>> rocksdb
> > > > >>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted (At
> > > least
> > > > >>>>>>> exposes an
> > > > >>>>>>>>>>>> API to
> > > > >>>>>>>>>>>>>>>>>>>           access the
> > > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted fashion).
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats are
> > well
> > > > >>>>>>> understood
> > > > >>>>>>>>> now
> > > > >>>>>>>>>>>> and we
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > basically
> > > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API Design
> (
> > > when
> > > > >> Adam
> > > > >>>>>>> sees
> > > > >>>>>>>>> my
> > > > >>>>>>>>>>>> newest
> > > > >>>>>>>>>>>>>>>>>>>           message).
> > > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of
> > > loosing
> > > > >>>>> those
> > > > >>>>>>> kinda
> > > > >>>>>>>>>>>>>>>>>>>           arguments within
> > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have no
> clue
> > > > why.
> > > > >> So
> > > > >>>>> I
> > > > >>>>>>>>>>>> literally
> > > > >>>>>>>>>>>>>>>>>>>           can't wait for
> > > > >>>>>>>>>>>>>>>>>>>           > you
> > > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and
> > give
> > > > you
> > > > >>>>>>> opinion on
> > > > >>>>>>>>>>>> how we
> > > > >>>>>>>>>>>>>>>>>>>           should
> > > > >>>>>>>>>>>>>>>>>>>           > design
> > > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the
> oneToManyJoin
> > > and
> > > > >> how
> > > > >>>>>>> many
> > > > >>>>>>>>>>>> power we
> > > > >>>>>>>>>>>>>>>>>>>           want to give
> > > > >>>>>>>>>>>>>>>>>>>           > to
> > > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where
> > > > simplicity
> > > > >>>>> isn't
> > > > >>>>>>>>>>>> really that
> > > > >>>>>>>>>>>>>>>>>>>           as users
> > > > >>>>>>>>>>>>>>>>>>>           > still
> > > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on the
> > > > >> discussion
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan
> > wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>
> > > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I
> found
> > > this
> > > > >>>>> subject
> > > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a join
> > > > table of
> > > > >>>>> two
> > > > >>>>>>>>>>>> streams? Is
> > > > >>>>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>           > somewhere
> > > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original
> requirement
> > or
> > > > >>>>> proposal?
> > > > >>>>>>>>>>>>>>>>>>>           > >>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan
> > > > Filipiak
> > > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
> > Bellemare
> > > > >> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a
> > > > Windowed
> > > > >>>>> Store
> > > > >>>>>>> to
> > > > >>>>>>>>>>>> store the
> > > > >>>>>>>>>>>>>>>>>>>           highwater
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this should
> > work
> > > > >> fine,
> > > > >>>>>>> with
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> caveat
> > > > >>>>>>>>>>>>>>>>>>>           being that
> > > > >>>>>>>>>>>>>>>>>>>           > it
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order
> arrival
> > > > for up
> > > > >>>>> to
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> size of
> > > > >>>>>>>>>>>>>>>>>>>           the window
> > > > >>>>>>>>>>>>>>>>>>>           > (ie:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would
> remove
> > > the
> > > > >>>>>>> possibility
> > > > >>>>>>>>>>>> of it
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> being
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > unbounded
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
> > suggestion, I
> > > > >>>>> believe
> > > > >>>>>>> this
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>>> where
> > > > >>>>>>>>>>>>>>>>>>>           we will
> > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement. While I
> > do
> > > > not
> > > > >>>>>>> disagree
> > > > >>>>>>>>>>>> with your
> > > > >>>>>>>>>>>>>>>>>>>           statement
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional
> > joins
> > > > done
> > > > >>>>> in a
> > > > >>>>>>>>>>>> real-world
> > > > >>>>>>>>>>>>>>>>>>>           workflow, I
> > > > >>>>>>>>>>>>>>>>>>>           > do
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively
> deal
> > > > with
> > > > >>>>>>>>> out-of-order
> > > > >>>>>>>>>>>>>>>>>>> arrival
> > > > >>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent joins. I
> > > have
> > > > >>>>>>> attempted
> > > > >>>>>>>>> what
> > > > >>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>           think you have
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a high-water,
> > > using
> > > > >>>>>>> groupBy and
> > > > >>>>>>>>>>>> reduce)
> > > > >>>>>>>>>>>>>>>>>>>           and found
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too
> > > quickly,
> > > > or
> > > > >>>>> the
> > > > >>>>>>> load
> > > > >>>>>>>>> on
> > > > >>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>           stream thread
> > > > >>>>>>>>>>>>>>>>>>>           > is
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages will
> > > arrive
> > > > >>>>>>>>> out-of-order
> > > > >>>>>>>>>>>> and be
> > > > >>>>>>>>>>>>>>>>>>>           incorrectly
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> > > intermediate
> > > > >>>>> event
> > > > >>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>> represented
> > > > >>>>>>>>>>>>>>>>>>>           as the
> > > > >>>>>>>>>>>>>>>>>>>           > final
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on your
> > > > groupBy
> > > > >>>>>>>>>>>> implementation.
> > > > >>>>>>>>>>>>>>>>>>>           There must be
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it is,
> I
> > > > would
> > > > >>>>> just
> > > > >>>>>>> like
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I
> would
> > > > like
> > > > >> to
> > > > >>>>>>> clarify
> > > > >>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>           we draw a
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the scattered
> > > events
> > > > >>>>> back to
> > > > >>>>>>>>> their
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> original
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I know
> > how
> > > > to
> > > > >>>>>>>>> conclusively
> > > > >>>>>>>>>>>> deal
> > > > >>>>>>>>>>>>>>>>>>>           with
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a given
> > time
> > > > >> frame,
> > > > >>>>>>> and to
> > > > >>>>>>>>>>>> ensure
> > > > >>>>>>>>>>>>>>>>>>>           that the
> > > > >>>>>>>>>>>>>>>>>>>           > data
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with the
> > > input
> > > > >>>>> events.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to share
> > that
> > > > >>>>>>> illustrates
> > > > >>>>>>>>> your
> > > > >>>>>>>>>>>>>>>>>>>           approach, I
> > > > >>>>>>>>>>>>>>>>>>>           > would
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would
> remove
> > > any
> > > > >>>>>>>>>>>> misunderstandings
> > > > >>>>>>>>>>>>>>>>>>>           that I may
> > > > >>>>>>>>>>>>>>>>>>>           > have.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for my
> > > code.
> > > > I
> > > > >>>>> don't
> > > > >>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>           something easily
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated
> with
> > > > >>>>> OO-patterns.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey, V
> > > > value, T
> > > > >>>>>>>>> aggregate)
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> > > currentStateAsMap =
> > > > >>>>>>>>>>>> asMap(aggregate);
> > > > >>>>>>>>>>>>>>>>>>> <<
> > > > >>>>>>>>>>>>>>>>>>>           imaginary
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> > > > >>>>> mapper.apply(value);
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the
> place
> > > > where
> > > > >>>>> people
> > > > >>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>           gonna have
> > > > >>>>>>>>>>>>>>>>>>>           > issues
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't do
> > it.
> > > > we
> > > > >>>>> would
> > > > >>>>>>> need
> > > > >>>>>>>>>>>> to find
> > > > >>>>>>>>>>>>>>>>>>>           a solution
> > > > >>>>>>>>>>>>>>>>>>>           > here.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate the
> > > > field in
> > > > >>>>> the
> > > > >>>>>>>>>>>> joiner, so
> > > > >>>>>>>>>>>>>>>>>>>           that we can
> > > > >>>>>>>>>>>>>>>>>>>           > pick
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate. Probably
> > you
> > > > have
> > > > >>>>> not
> > > > >>>>>>>>> thought
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>           this in your
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open
> to
> > > > find a
> > > > >>>>>>> generic
> > > > >>>>>>>>>>>> solution
> > > > >>>>>>>>>>>>>>>>>>>           here. In my
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken in
> > > > >>>>>>>>> KTableImpl.GroupBy
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>           looses
> > > > >>>>>>>>>>>>>>>>>>>           > the keys
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the aggregate
> > > key.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted it
> > away
> > > > back
> > > > >>>>>>> then way
> > > > >>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>> i
> > > > >>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>           > thinking
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is why I
> > > > didn't
> > > > >>>>>>> realize
> > > > >>>>>>>>> its
> > > > >>>>>>>>>>>>>>>>>>>           significance here.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > currentStateAsMap.put(mapper.apply(m),
> > > > >> m);
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> currentStateAsMap.put(toModifyKey,
> > > > >> value);
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > currentStateAsMap.remove(toModifyKey);
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> if(currentStateAsMap.isEmpty()){
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> > > > >>>>>>> asAggregateType(currentStateAsMap)
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35 PM,
> > Jan
> > > > >>>>> Filipiak
> > > > >>>>>>> <
> > > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
> > > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing
> Matthias
> > > to
> > > > >>>>> speed!
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I think
> > > > >> re-keying
> > > > >>>>>>> back
> > > > >>>>>>>>>>>> should be
> > > > >>>>>>>>>>>>>>>>>>>           optional at
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> > > > KScatteredTable
> > > > >>>>> with
> > > > >>>>>>>>>>>> reshuffle()
> > > > >>>>>>>>>>>>>>>>>>>           returning
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined> to
> > make
> > > > the
> > > > >>>>>>> backwards
> > > > >>>>>>>>>>>>>>>>>>>           repartitioning
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of
> > doing
> > > > the
> > > > >>>>> out
> > > > >>>>>>> of
> > > > >>>>>>>>> order
> > > > >>>>>>>>>>>>>>>>>>>           processing using
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark
> > > tracking.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded growth
> is
> > > > just
> > > > >>>>> scary
> > > > >>>>>>> + It
> > > > >>>>>>>>>>>> saves
> > > > >>>>>>>>>>>>>>>>>>> us
> > > > >>>>>>>>>>>>>>>>>>>           the header
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of
> > always
> > > > >>>>>>> repartitioning
> > > > >>>>>>>>>>>> back is
> > > > >>>>>>>>>>>>>>>>>>>           just not so
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has been
> > > done
> > > > >>>>> before
> > > > >>>>>>> we
> > > > >>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>           back and
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards
> is
> > > > really
> > > > >>>>>>> common.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
> > > Bellemare
> > > > >>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your feedback,
> I
> > do
> > > > >>>>>>> appreciate
> > > > >>>>>>>>> it!
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be
> > > > possible,
> > > > >> it
> > > > >>>>>>> would
> > > > >>>>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           > deserialize
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > > > runtime
> > > > >>>>>>> overhead.
> > > > >>>>>>>>> I
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid
> the
> > > > >>>>> overhead.
> > > > >>>>>>> If
> > > > >>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> > > name
> > > > >>>>> spacing
> > > > >>>>>>>>> later
> > > > >>>>>>>>>>>> on.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with
> using a
> > > > >> reserved
> > > > >>>>>>> string
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>           document it.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the
> > design
> > > it
> > > > >> the
> > > > >>>>>>> type of
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
> > > correctly,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have
> > table1
> > > > and
> > > > >>>>> table2
> > > > >>>>>>>>>>>> swapped.
> > > > >>>>>>>>>>>>>>>>>>>           Here is how it
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records
> that
> > > > contain
> > > > >>>>> the
> > > > >>>>>>>>>>>> foreign key
> > > > >>>>>>>>>>>>>>>>>>>           within their
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> > > > <a,(fk=A,bar=1)>,
> > > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>,
> > <B,Y>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is required
> > to
> > > > >> extract
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> foreign
> > > > >>>>>>>>>>>>>>>>>>> key.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper: (
> > > value
> > > > =>
> > > > >>>>>>> value.fk
> > > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to each
> > > > element
> > > > >> in
> > > > >>>>>>>>> table1,
> > > > >>>>>>>>>>>> and a
> > > > >>>>>>>>>>>>>>>>>>>           new combined
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> > > (fk=A,bar=1)>,
> > > > >>>>> <A-b,
> > > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > > > >>>>>>>>>>>>>>>>>>>           <B-c,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> > > > >> copartitioned
> > > > >>>>>>> with
> > > > >>>>>>>>>>>> table2:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with
> Partition
> > > 0:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
> > > > >>>>> (fk=A,bar=1)>,
> > > > >>>>>>> <A-b,
> > > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with
> Partition
> > > 1:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
> > > > >> (fk=B,bar=3)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be
> > joined
> > > > >>>>> together
> > > > >>>>>>>>> locally
> > > > >>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>>>           applying the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design
> and
> > > my
> > > > >>>>> design
> > > > >>>>>>>>>>>> deviate. My
> > > > >>>>>>>>>>>>>>>>>>>           design goes
> > > > >>>>>>>>>>>>>>>>>>>           > on
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data
> post-join
> > > and
> > > > >>>>> resolve
> > > > >>>>>>>>>>>> out-of-order
> > > > >>>>>>>>>>>>>>>>>>>           arrival of
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data
> > keyed
> > > > just
> > > > >>>>> the
> > > > >>>>>>>>>>>> original key.
> > > > >>>>>>>>>>>>>>>>>>>           I do not
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the
> > > internals
> > > > >>>>>>> outside of
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make for
> > > larger
> > > > >>>>>>> footprint,
> > > > >>>>>>>>>>>> but it
> > > > >>>>>>>>>>>>>>>>>>>           removes all
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order
> > > arrivals
> > > > >> and
> > > > >>>>>>>>> handling
> > > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> > > > >>>>>>>>>>>>>>>>>>>           > the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this
> makes
> > > the
> > > > >>>>>>> function
> > > > >>>>>>>>> much
> > > > >>>>>>>>>>>>>>>>>>> easier
> > > > >>>>>>>>>>>>>>>>>>>           to use.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps
> > resolve
> > > > your
> > > > >>>>>>>>> questions,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>           please feel
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your
> mind.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36
> PM,
> > > > >>>>> Matthias J.
> > > > >>>>>>>>> Sax <
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io
> <mailto:
> > > > >>>>>>>>>>>> matthias@confluent.io>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on
> this
> > > > >> thread. I
> > > > >>>>>>> did
> > > > >>>>>>>>> not
> > > > >>>>>>>>>>>> read
> > > > >>>>>>>>>>>>>>>>>>>           everything so
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple of
> > > > initial
> > > > >>>>>>> thoughts:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is a
> > > > >> fundamental
> > > > >>>>>>>>>>>> difference
> > > > >>>>>>>>>>>>>>>>>>>           between header
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258. For
> > 258,
> > > > we
> > > > >> add
> > > > >>>>>>>>> headers
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams
> and
> > > > nobody
> > > > >>>>>>> else is
> > > > >>>>>>>>>>>> supposed
> > > > >>>>>>>>>>>>>>>>>>>           to write
> > > > >>>>>>>>>>>>>>>>>>>           > into
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user
> header
> > > are
> > > > >>>>> written
> > > > >>>>>>> into
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not
> conflicts.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see a
> > big
> > > > issue
> > > > >>>>> with
> > > > >>>>>>>>> using
> > > > >>>>>>>>>>>>>>>>>>>           headers within
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it,
> we
> > > can
> > > > >> have
> > > > >>>>>>> some
> > > > >>>>>>>>>>>> "reserved"
> > > > >>>>>>>>>>>>>>>>>>>           header keys
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to use
> > > when
> > > > >>>>>>> processing
> > > > >>>>>>>>>>>> data with
> > > > >>>>>>>>>>>>>>>>>>>           Kafka
> > > > >>>>>>>>>>>>>>>>>>>           > Streams.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe way
> to
> > > > avoid
> > > > >>>>>>>>> conflicts,
> > > > >>>>>>>>>>>> since
> > > > >>>>>>>>>>>>>>>>>>> these
> > > > >>>>>>>>>>>>>>>>>>>           > headers
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal
> > topics
> > > (I
> > > > >>>>> think):
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and changelog
> > > > topics,
> > > > >> we
> > > > >>>>> can
> > > > >>>>>>>>>>>> namespace
> > > > >>>>>>>>>>>>>>>>>>>           all headers:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers are
> > > > >> namespaced
> > > > >>>>> as
> > > > >>>>>>>>>>>> "external."
> > > > >>>>>>>>>>>>>>>>>>> +
> > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> > > > namespaced as
> > > > >>>>>>>>>>>> "internal." +
> > > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would be
> > > > >> possible,
> > > > >>>>> it
> > > > >>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > > > runtime
> > > > >>>>>>> overhead.
> > > > >>>>>>>>> I
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > > >>>>>>>>>>>>>>>>>>>           > no
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid
> the
> > > > >>>>> overhead.
> > > > >>>>>>> If
> > > > >>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> becomes a
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > problem in
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> > > name
> > > > >>>>> spacing
> > > > >>>>>>>>> later
> > > > >>>>>>>>>>>> on.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the
> > design
> > > > it
> > > > >> the
> > > > >>>>>>> type
> > > > >>>>>>>>> of
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
> > > correctly,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable =
> > > > >>>>>>>>>>>> table1.join(table2,...);
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the
> `joinedTable`
> > > has
> > > > >> the
> > > > >>>>>>> same
> > > > >>>>>>>>> key
> > > > >>>>>>>>>>>> as the
> > > > >>>>>>>>>>>>>>>>>>>           left input
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work
> > because
> > > > if
> > > > >>>>> table2
> > > > >>>>>>>>>>>> contains
> > > > >>>>>>>>>>>>>>>>>>>           multiple rows
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in table1
> > > > (what is
> > > > >>>>> the
> > > > >>>>>>> main
> > > > >>>>>>>>>>>> purpose
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>           > foreign
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table
> would
> > > only
> > > > >>>>>>> contain a
> > > > >>>>>>>>>>>> single
> > > > >>>>>>>>>>>>>>>>>>>           join result,
> > > > >>>>>>>>>>>>>>>>>>>           > but
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
> > <a,(A,1)>,
> > > > >>>>> <b,(A,2)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a
> foreign
> > > key
> > > > to
> > > > >>>>>>> table1
> > > > >>>>>>>>> key
> > > > >>>>>>>>>>>> (ie,
> > > > >>>>>>>>>>>>>>>>>>>           "A" joins).
> > > > >>>>>>>>>>>>>>>>>>>           > If
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key
> as
> > > key
> > > > of
> > > > >>>>>>> table1,
> > > > >>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>           implies that the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> > > > join(X,1)>
> > > > >> or
> > > > >>>>> <A,
> > > > >>>>>>>>>>>> join(X,2)>
> > > > >>>>>>>>>>>>>>>>>>>           but not
> > > > >>>>>>>>>>>>>>>>>>>           > both.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same
> > key,
> > > > >>>>> whatever
> > > > >>>>>>>>> result
> > > > >>>>>>>>>>>> record
> > > > >>>>>>>>>>>>>>>>>>>           we emit
> > > > >>>>>>>>>>>>>>>>>>>           > later,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous
> result.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan
> > > > originally
> > > > >>>>>>> proposed
> > > > >>>>>>>>>>>> to use
> > > > >>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>           > combination
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the
> input
> > > > tables
> > > > >>>>> as
> > > > >>>>>>> key
> > > > >>>>>>>>> of
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           output table.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the output
> > > table
> > > > >>>>> unique
> > > > >>>>>>> and
> > > > >>>>>>>>> we
> > > > >>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>           store both in
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> > > join(X,1)>,
> > > > >> <A-b,
> > > > >>>>>>>>>>>> join(X,2)>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan
> > Filipiak
> > > > >> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could be
> > > > >>>>> disregarded.
> > > > >>>>>>> The
> > > > >>>>>>>>>>>> decision
> > > > >>>>>>>>>>>>>>>>>>>           about the
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of the
> > > > >> aggregated
> > > > >>>>>>> map.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps
> > would
> > > be
> > > > >>>>>>> unpacked
> > > > >>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> > > > >>>>>>>>>>>>>>>>>>>           > element
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as
> delete.
> > > Any
> > > > >>>>> other
> > > > >>>>>>> count
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in
> "waiting
> > > for
> > > > >>>>> correct
> > > > >>>>>>>>>>>> deletes to
> > > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam
> > > > Bellemare
> > > > >>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could
> > > replace
> > > > >> the
> > > > >>>>>>> second
> > > > >>>>>>>>>>>>>>>>>>>           repartition store
> > > > >>>>>>>>>>>>>>>>>>>           > and
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a
> > groupBy
> > > > and
> > > > >>>>>>> reduce.
> > > > >>>>>>>>>>>> However,
> > > > >>>>>>>>>>>>>>>>>>>           it looks
> > > > >>>>>>>>>>>>>>>>>>>           > like
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the
> > > > highwater
> > > > >>>>> value
> > > > >>>>>>>>> within
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           materialized
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> > > > out-of-order
> > > > >>>>>>> records
> > > > >>>>>>>>>>>> (assuming
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> my
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This
> in
> > > > effect
> > > > >> is
> > > > >>>>>>> the
> > > > >>>>>>>>> same
> > > > >>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>           design I
> > > > >>>>>>>>>>>>>>>>>>>           > have
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables
> > merged
> > > > >>>>> together.
> > > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>           >
> > > > >>>>>>>>>>>>>>>>>>>           >
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>> -- Guozhang
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>> -- Guozhang
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>


-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Adam,

Given that the committers are all pretty busy right now, I think that it
would help if you were to refactor the KIP a little to reduce the workload
for reviewers.

I'd recommend the following changes:
* relocate all internal details to a section at the end called something
like "Implementation Notes" or something like that.
* rewrite the rest of the KIP to be a succinct as possible and mention only
publicly-facing API changes.
** for example, the interface that you've already listed there, as well as
a textual description of the guarantees we'll be providing (join result is
copartitioned with the LHS, and the join result is guaranteed correct)

A good target would be that the whole main body of the KIP, including
Status, Motivation, Proposal, Justification, and Rejected Alternatives all
fit "above the fold" (i.e., all fit on the screen at a comfortable zoom
level).
I think the only real Rejected Alternative that bears mention at this point
is KScatteredTable, which you could just include the executive summary on
(no implementation details), and link to extra details in the
Implementation Notes section.

Taking a look at the wiki page, ~90% of the text there is internal detail,
which is useful for the dubious, but doesn't need to be ratified in a vote
(and would be subject to change without notice in the future anyway).
There's also a lot of conflicting discussion, as you've very respectfully
tried to preserve the original proposal from Jan while adding your own.
Isolating all this information in a dedicated section at the bottom frees
the voters up to focus on the public API part of the proposal, which is
really all they need to consider.

Plus, it'll be clear to future readers which parts of the document are
enduring, and which parts are a snapshot of our implementation thinking at
the time.

I'm suggesting this because I suspect that the others haven't made time to
review it partly because it seems daunting. If it seems like it would be a
huge time investment to review, people will just keep putting it off. But
if the KIP is a single page, then they'll be more inclined to give it a
read.

Honestly, I don't think the KIP itself is that controversial (apart from
the scattered table thing (sorry, Jan) ). Most of the discussion has been
around the implementation, which we can continue more effectively in a PR
once the KIP has passed.

How does that sound?
-John

On Mon, Dec 10, 2018 at 3:54 PM Adam Bellemare <ad...@gmail.com>
wrote:

> 1) I believe that the resolution mechanism John has proposed is sufficient
> - it is clean and easy and doesn't require additional RocksDB stores, which
> reduces the footprint greatly. I don't think we need to resolve based on
> timestamp or offset anymore, but if we decide to do to that would be within
> the bounds of the existing API.
>
> 2) Is the current API sufficient, or does it need to be altered to go back
> to vote?
>
> 3) KScatteredTable implementation can always be added in a future revision.
> This API does not rule it out. This implementation of this function would
> simply be replaced with `KScatteredTable.resolve()` while still maintaining
> the existing API, thereby giving both features as Jan outlined earlier.
> Would this work?
>
>
> Thanks Guozhang, John and Jan
>
>
>
>
> On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io> wrote:
>
> > Hi, all,
> >
> > >> In fact, we
> > >> can just keep a single final-result store with timestamps and reject
> > values
> > >> that have a smaller timestamp, is that right?
> >
> > > Which is the correct output should at least be decided on the offset of
> > > the original message.
> >
> > Thanks for this point, Jan.
> >
> > KIP-258 is merely to allow embedding the record timestamp  in the k/v
> > store,
> > as well as providing a storage-format upgrade path.
> >
> > I might have missed it, but I think we have yet to discuss whether it's
> > safe
> > or desirable just to swap topic-ordering our for timestamp-ordering. This
> > is
> > a very deep topic, and I think it would only pollute the current
> > discussion.
> >
> > What Adam has proposed is safe, given the *current* ordering semantics
> > of the system. If we can agree on his proposal, I think we can merge the
> > feature well before the conversation about timestamp ordering even takes
> > place, much less reaches a conclusion. In the mean time, it would seem to
> > be unfortunate to have one join operator with different ordering
> semantics
> > from every other KTable operator.
> >
> > If and when that timestamp discussion takes place, many (all?) KTable
> > operations
> > will need to be updated, rendering the many:one join a small marginal
> cost.
> >
> > And, just to plug it again, I proposed an algorithm above that I believe
> > provides
> > correct ordering without any additional metadata, and regardless of the
> > ordering semantics. I didn't bring it up further, because I felt the KIP
> > only needs
> > to agree on the public API, and we can discuss the implementation at
> > leisure in
> > a PR...
> >
> > Thanks,
> > -John
> >
> >
> > On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> > >
> > >
> > > On 10.12.2018 07:42, Guozhang Wang wrote:
> > > > Hello Adam / Jan / John,
> > > >
> > > > Sorry for being late on this thread! I've finally got some time this
> > > > weekend to cleanup a load of tasks on my queue (actually I've also
> > > realized
> > > > there are a bunch of other things I need to enqueue while cleaning
> them
> > > up
> > > > --- sth I need to improve on my side). So here are my thoughts:
> > > >
> > > > Regarding the APIs: I like the current written API in the KIP. More
> > > > generally I'd prefer to keep the 1) one-to-many join functionalities
> as
> > > > well as 2) other join types than inner as separate KIPs since 1) may
> > > worth
> > > > a general API refactoring that can benefit not only foreignkey joins
> > but
> > > > collocate joins as well (e.g. an extended proposal of
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > ),
> > > > and I'm not sure if other join types would actually be needed (maybe
> > left
> > > > join still makes sense), so it's better to
> > wait-for-people-to-ask-and-add
> > > > than add-sth-that-no-one-uses.
> > > >
> > > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > > > KScatteredTable for users to inject their own optimization: I'd
> prefer
> > to
> > > > do the current option as-is, and my main rationale is for
> optimization
> > > > rooms inside the Streams internals and the API succinctness. For
> > advanced
> > > > users who may indeed prefer KScatteredTable and do their own
> > > optimization,
> > > > while it is too much of the work to use Processor API directly, I
> think
> > > we
> > > > can still extend the current API to support it in the future if it
> > > becomes
> > > > necessary.
> > >
> > > no internal optimization potential. it's a myth
> > >
> > > ¯\_(ツ)_/¯
> > >
> > > :-)
> > >
> > > >
> > > > Another note about step 4) resolving out-of-ordering data, as I
> > mentioned
> > > > before I think with KIP-258 (embedded timestamp with key-value store)
> > we
> > > > can actually make this step simpler than the current proposal. In
> fact,
> > > we
> > > > can just keep a single final-result store with timestamps and reject
> > > values
> > > > that have a smaller timestamp, is that right?
> > >
> > > Which is the correct output should at least be decided on the offset of
> > > the original message.
> > >
> > > >
> > > >
> > > > That's all I have in mind now. Again, great appreciation to Adam to
> > make
> > > > such HUGE progress on this KIP!
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <
> Jan.Filipiak@trivago.com>
> > > > wrote:
> > > >
> > > >> If they don't find the time:
> > > >> They usually take the opposite path from me :D
> > > >> so the answer would be clear.
> > > >>
> > > >> hence my suggestion to vote.
> > > >>
> > > >>
> > > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> > > >>> Hi Guozhang and Matthias
> > > >>>
> > > >>> I know both of you are quite busy, but we've gotten this KIP to a
> > point
> > > >>> where we need more guidance on the API (perhaps a bit of a
> > tie-breaker,
> > > >> if
> > > >>> you will). If you have anyone else you may think should look at
> this,
> > > >>> please tag them accordingly.
> > > >>>
> > > >>> The scenario is as such:
> > > >>>
> > > >>> Current Option:
> > > >>> API:
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > > >>> 1) Rekey the data to CombinedKey, and shuffles it to the partition
> > with
> > > >> the
> > > >>> foreignKey (repartition 1)
> > > >>> 2) Join the data
> > > >>> 3) Shuffle the data back to the original node (repartition 2)
> > > >>> 4) Resolve out-of-order arrival / race condition due to foreign-key
> > > >> changes.
> > > >>>
> > > >>> Alternate Option:
> > > >>> Perform #1 and #2 above, and return a KScatteredTable.
> > > >>> - It would be keyed on a wrapped key function: <CombinedKey<KO, K>,
> > VR>
> > > >> (KO
> > > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> > > >>> - KScatteredTable.resolve() would perform #3 and #4 but otherwise a
> > > user
> > > >>> would be able to perform additional functions directly from the
> > > >>> KScatteredTable (TBD - currently out of scope).
> > > >>> - John's analysis 2-emails up is accurate as to the tradeoffs.
> > > >>>
> > > >>> Current Option is coded as-is. Alternate option is possible, but
> will
> > > >>> require for implementation details to be made in the API and some
> > > >> exposure
> > > >>> of new data structures into the API (ie: CombinedKey).
> > > >>>
> > > >>> I appreciate any insight into this.
> > > >>>
> > > >>> Thanks.
> > > >>>
> > > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > > adam.bellemare@gmail.com>
> > > >>> wrote:
> > > >>>
> > > >>>> Hi John
> > > >>>>
> > > >>>> Thanks for your feedback and assistance. I think your summary is
> > > >> accurate
> > > >>>> from my perspective. Additionally, I would like to add that there
> > is a
> > > >> risk
> > > >>>> of inconsistent final states without performing the resolution.
> This
> > > is
> > > >> a
> > > >>>> major concern for me as most of the data I have dealt with is
> > produced
> > > >> by
> > > >>>> relational databases. We have seen a number of cases where a user
> in
> > > the
> > > >>>> Rails UI has modified the field (foreign key), realized they made
> a
> > > >>>> mistake, and then updated the field again with a new key. The
> events
> > > are
> > > >>>> propagated out as they are produced, and as such we have had
> > > real-world
> > > >>>> cases where these inconsistencies were propagated downstream as
> the
> > > >> final
> > > >>>> values due to the race conditions in the fanout of the data.
> > > >>>>
> > > >>>> This solution that I propose values correctness of the final
> result
> > > over
> > > >>>> other factors.
> > > >>>>
> > > >>>> We could always move this function over to using a KScatteredTable
> > > >>>> implementation in the future, and simply deprecate it this join
> API
> > in
> > > >>>> time. I think I would like to hear more from some of the other
> major
> > > >>>> committers on which course of action they would think is best
> before
> > > any
> > > >>>> more coding is done.
> > > >>>>
> > > >>>> Thanks again
> > > >>>>
> > > >>>> Adam
> > > >>>>
> > > >>>>
> > > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io>
> > > wrote:
> > > >>>>
> > > >>>>> Hi Jan and Adam,
> > > >>>>>
> > > >>>>> Wow, thanks for doing that test, Adam. Those results are
> > encouraging.
> > > >>>>>
> > > >>>>> Thanks for your performance experience as well, Jan. I agree that
> > > >> avoiding
> > > >>>>> unnecessary join outputs is especially important when the fan-out
> > is
> > > so
> > > >>>>> high. I suppose this could also be built into the implementation
> > > we're
> > > >>>>> discussing, but it wouldn't have to be specified in the KIP
> (since
> > > >> it's an
> > > >>>>> API-transparent optimization).
> > > >>>>>
> > > >>>>> As far as whether or not to re-repartition the data, I didn't
> bring
> > > it
> > > >> up
> > > >>>>> because it sounded like the two of you agreed to leave the KIP
> > as-is,
> > > >>>>> despite the disagreement.
> > > >>>>>
> > > >>>>> If you want my opinion, I feel like both approaches are
> reasonable.
> > > >>>>> It sounds like Jan values more the potential for developers to
> > > optimize
> > > >>>>> their topologies to re-use the intermediate nodes, whereas Adam
> > > places
> > > >>>>> more
> > > >>>>> value on having a single operator that people can use without
> extra
> > > >> steps
> > > >>>>> at the end.
> > > >>>>>
> > > >>>>> Personally, although I do find it exceptionally annoying when a
> > > >> framework
> > > >>>>> gets in my way when I'm trying to optimize something, it seems
> > better
> > > >> to
> > > >>>>> go
> > > >>>>> for a single operation.
> > > >>>>> * Encapsulating the internal transitions gives us significant
> > > latitude
> > > >> in
> > > >>>>> the implementation (for example, joining only at the end, not in
> > the
> > > >>>>> middle
> > > >>>>> to avoid extra data copying and out-of-order resolution; how we
> > > >> represent
> > > >>>>> the first repartition keys (combined keys vs. value vectors),
> > etc.).
> > > >> If we
> > > >>>>> publish something like a KScatteredTable with the
> right-partitioned
> > > >> joined
> > > >>>>> data, then the API pretty much locks in the implementation as
> well.
> > > >>>>> * The API seems simpler to understand and use. I do mean "seems";
> > if
> > > >>>>> anyone
> > > >>>>> wants to make the case that KScatteredTable is actually simpler,
> I
> > > >> think
> > > >>>>> hypothetical usage code would help. From a relational algebra
> > > >> perspective,
> > > >>>>> it seems like KTable.join(KTable) should produce a new KTable in
> > all
> > > >>>>> cases.
> > > >>>>> * That said, there might still be room in the API for a different
> > > >>>>> operation
> > > >>>>> like what Jan has proposed to scatter a KTable, and then do
> things
> > > like
> > > >>>>> join, re-group, etc from there... I'm not sure; I haven't thought
> > > >> through
> > > >>>>> all the consequences yet.
> > > >>>>>
> > > >>>>> This is all just my opinion after thinking over the discussion so
> > > >> far...
> > > >>>>> -John
> > > >>>>>
> > > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > > >> adam.bellemare@gmail.com>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> Updated the PR to take into account John's feedback.
> > > >>>>>>
> > > >>>>>> I did some preliminary testing for the performance of the
> > > prefixScan.
> > > >> I
> > > >>>>>> have attached the file, but I will also include the text in the
> > body
> > > >>>>> here
> > > >>>>>> for archival purposes (I am not sure what happens to attached
> > > files).
> > > >> I
> > > >>>>>> also updated the PR and the KIP accordingly.
> > > >>>>>>
> > > >>>>>> Summary: It scales exceptionally well for scanning large values
> of
> > > >>>>>> records. As Jan mentioned previously, the real issue would be
> more
> > > >>>>> around
> > > >>>>>> processing the resulting records after obtaining them. For
> > instance,
> > > >> it
> > > >>>>>> takes approximately ~80-120 mS to flush the buffer and a further
> > > >>>>> ~35-85mS
> > > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them.
> > Iterating
> > > >>>>>> through the records just to generate a simple count takes ~ 40
> > times
> > > >>>>> longer
> > > >>>>>> than the flush + scan combined.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> ============================================================================================
> > > >>>>>> Setup:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> ============================================================================================
> > > >>>>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m,
> > > >> Xms512m)
> > > >>>>>> CPU: i7 2.2 Ghz.
> > > >>>>>>
> > > >>>>>> Note: I am using a slightly-modified, directly-accessible Kafka
> > > >> Streams
> > > >>>>>> RocksDB
> > > >>>>>> implementation (RocksDB.java, basically just avoiding the
> > > >>>>>> ProcessorContext).
> > > >>>>>> There are no modifications to the default RocksDB values
> provided
> > in
> > > >> the
> > > >>>>>> 2.1/trunk release.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> keysize = 128 bytes
> > > >>>>>> valsize = 512 bytes
> > > >>>>>>
> > > >>>>>> Step 1:
> > > >>>>>> Write X positive matching events: (key = prefix + left-padded
> > > >>>>>> auto-incrementing integer)
> > > >>>>>> Step 2:
> > > >>>>>> Write 10X negative matching events (key = left-padded
> > > >> auto-incrementing
> > > >>>>>> integer)
> > > >>>>>> Step 3:
> > > >>>>>> Perform flush
> > > >>>>>> Step 4:
> > > >>>>>> Perform prefixScan
> > > >>>>>> Step 5:
> > > >>>>>> Iterate through return Iterator and validate the count of
> expected
> > > >>>>> events.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> ============================================================================================
> > > >>>>>> Results:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> ============================================================================================
> > > >>>>>> X = 1k (11k events total)
> > > >>>>>> Flush Time = 39 mS
> > > >>>>>> Scan Time = 7 mS
> > > >>>>>> 6.9 MB disk
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> --------------------------------------------------------------------------------------------
> > > >>>>>> X = 10k (110k events total)
> > > >>>>>> Flush Time = 45 mS
> > > >>>>>> Scan Time = 8 mS
> > > >>>>>> 127 MB
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> --------------------------------------------------------------------------------------------
> > > >>>>>> X = 100k (1.1M events total)
> > > >>>>>> Test1:
> > > >>>>>> Flush Time = 60 mS
> > > >>>>>> Scan Time = 12 mS
> > > >>>>>> 678 MB
> > > >>>>>>
> > > >>>>>> Test2:
> > > >>>>>> Flush Time = 45 mS
> > > >>>>>> Scan Time = 7 mS
> > > >>>>>> 576 MB
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> --------------------------------------------------------------------------------------------
> > > >>>>>> X = 1MB (11M events total)
> > > >>>>>> Test1:
> > > >>>>>> Flush Time = 52 mS
> > > >>>>>> Scan Time = 19 mS
> > > >>>>>> 7.2 GB
> > > >>>>>>
> > > >>>>>> Test2:
> > > >>>>>> Flush Time = 84 mS
> > > >>>>>> Scan Time = 34 mS
> > > >>>>>> 9.1 GB
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> --------------------------------------------------------------------------------------------
> > > >>>>>> X = 2.5M (27.5M events total)
> > > >>>>>> Test1:
> > > >>>>>> Flush Time = 82 mS
> > > >>>>>> Scan Time = 63 mS
> > > >>>>>> 17GB - 276 sst files
> > > >>>>>>
> > > >>>>>> Test2:
> > > >>>>>> Flush Time = 116 mS
> > > >>>>>> Scan Time = 35 mS
> > > >>>>>> 23GB - 361 sst files
> > > >>>>>>
> > > >>>>>> Test3:
> > > >>>>>> Flush Time = 103 mS
> > > >>>>>> Scan Time = 82 mS
> > > >>>>>> 19 GB - 300 sst files
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>
> > >
> >
> --------------------------------------------------------------------------------------------
> > > >>>>>>
> > > >>>>>> I had to limit my testing on my laptop to X = 2.5M events. I
> tried
> > > to
> > > >> go
> > > >>>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+
> > range
> > > >>>>> and my
> > > >>>>>> laptop ran out of disk. More extensive testing could be done
> but I
> > > >>>>> suspect
> > > >>>>>> that it would be in line with what we're seeing in the results
> > > above.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> At this point in time, I think the only major discussion point
> is
> > > >> really
> > > >>>>>> around what Jan and I have disagreed on: repartitioning back +
> > > >> resolving
> > > >>>>>> potential out of order issues or leaving that up to the client
> to
> > > >>>>> handle.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Thanks folks,
> > > >>>>>>
> > > >>>>>> Adam
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > > Jan.Filipiak@trivago.com
> > > >>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > > >>>>>>>> Hi all,
> > > >>>>>>>>
> > > >>>>>>>> Sorry that this discussion petered out... I think the 2.1
> > release
> > > >>>>>>> caused an
> > > >>>>>>>> extended distraction that pushed it off everyone's radar
> (which
> > > was
> > > >>>>>>>> precisely Adam's concern). Personally, I've also had some
> extend
> > > >>>>>>>> distractions of my own that kept (and continue to keep) me
> > > >>>>> preoccupied.
> > > >>>>>>>>
> > > >>>>>>>> However, calling for a vote did wake me up, so I guess Jan was
> > on
> > > >> the
> > > >>>>>>> right
> > > >>>>>>>> track!
> > > >>>>>>>>
> > > >>>>>>>> I've gone back and reviewed the whole KIP document and the
> prior
> > > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> > > >>>>>>>>
> > > >>>>>>>> API Thoughts:
> > > >>>>>>>>
> > > >>>>>>>> 1. If I read the KIP right, you are proposing a many-to-one
> > join.
> > > >>>>> Could
> > > >>>>>>> we
> > > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the
> > > design
> > > >>>>>>> around
> > > >>>>>>>> and make it a oneToManyJoin?
> > > >>>>>>>>
> > > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join type,
> > and
> > > it
> > > >>>>>>> seems
> > > >>>>>>>> like it might trick some people into using it for a one-to-one
> > > join.
> > > >>>>>>> This
> > > >>>>>>>> would work, of course, but it would be super inefficient
> > compared
> > > to
> > > >>>>> a
> > > >>>>>>>> simple rekey-and-join.
> > > >>>>>>>>
> > > >>>>>>>> 2. I might have missed it, but I don't think it's specified
> > > whether
> > > >>>>>>> it's an
> > > >>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
> > > >>>>> (neglecting
> > > >>>>>>> IQ),
> > > >>>>>>>> the rest can be achieved by filtering or by handling it in the
> > > >>>>>>> ValueJoiner.
> > > >>>>>>>>
> > > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
> > > >>>>>>>> 3a. Regarding Serialized: There are a few different paradigms
> in
> > > >>>>> play in
> > > >>>>>>>> the Streams API, so it's confusing, but instead of three
> > > Serialized
> > > >>>>>>> args, I
> > > >>>>>>>> think it would be better to have one that allows (optionally)
> > > >> setting
> > > >>>>>>> the 4
> > > >>>>>>>> incoming serdes. The result serde is defined by the
> > Materialized.
> > > >> The
> > > >>>>>>>> incoming serdes can be optional because they might already be
> > > >>>>> available
> > > >>>>>>> on
> > > >>>>>>>> the source KTables, or the default serdes from the config
> might
> > be
> > > >>>>>>>> applicable.
> > > >>>>>>>>
> > > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't
> > > allow
> > > >>>>>>> setting
> > > >>>>>>>> one, and it seems like it might actually be harmful, since the
> > > rekey
> > > >>>>>>>> operation needs to produce results that are co-partitioned
> with
> > > the
> > > >>>>>>> "other"
> > > >>>>>>>> KTable.
> > > >>>>>>>>
> > > >>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't
> > actually
> > > >>>>>>> follow
> > > >>>>>>>> what Matthias meant about namespacing requiring
> "deserializing"
> > > the
> > > >>>>>>> record
> > > >>>>>>>> header. The headers are already Strings, so I don't think that
> > > >>>>>>>> deserialization is required. If we applied the namespace at
> > source
> > > >>>>> nodes
> > > >>>>>>>> and stripped it at sink nodes, this would be practically no
> > > >> overhead.
> > > >>>>>>> The
> > > >>>>>>>> advantage of the namespace idea is that no public API change
> wrt
> > > >>>>> headers
> > > >>>>>>>> needs to happen, and no restrictions need to be placed on
> users'
> > > >>>>>>> headers.
> > > >>>>>>>>
> > > >>>>>>>> (Although I'm wondering if we can get away without the header
> at
> > > >>>>> all...
> > > >>>>>>>> stay tuned)
> > > >>>>>>>>
> > > >>>>>>>> 5. I also didn't follow the discussion about the HWM table
> > growing
> > > >>>>>>> without
> > > >>>>>>>> bound. As I read it, the HWM table is effectively implementing
> > OCC
> > > >> to
> > > >>>>>>>> resolve the problem you noted with disordering when the rekey
> is
> > > >>>>>>>> reversed... particularly notable when the FK changes. As such,
> > it
> > > >>>>> only
> > > >>>>>>>> needs to track the most recent "version" (the offset in the
> > source
> > > >>>>>>>> partition) of each key. Therefore, it should have the same
> > number
> > > of
> > > >>>>>>> keys
> > > >>>>>>>> as the source table at all times.
> > > >>>>>>>>
> > > >>>>>>>> I see that you are aware of KIP-258, which I think might be
> > > relevant
> > > >>>>> in
> > > >>>>>>> a
> > > >>>>>>>> couple of ways. One: it's just about storing the timestamp in
> > the
> > > >>>>> state
> > > >>>>>>>> store, but the ultimate idea is to effectively use the
> timestamp
> > > as
> > > >>>>> an
> > > >>>>>>> OCC
> > > >>>>>>>> "version" to drop disordered updates. You wouldn't want to use
> > the
> > > >>>>>>>> timestamp for this operation, but if you were to use a similar
> > > >>>>>>> mechanism to
> > > >>>>>>>> store the source offset in the store alongside the re-keyed
> > > values,
> > > >>>>> then
> > > >>>>>>>> you could avoid a separate table.
> > > >>>>>>>>
> > > >>>>>>>> 6. You and Jan have been thinking about this for a long time,
> so
> > > >> I've
> > > >>>>>>>> probably missed something here, but I'm wondering if we can
> > avoid
> > > >> the
> > > >>>>>>> HWM
> > > >>>>>>>> tracking at all and resolve out-of-order during a final join
> > > >>>>> instead...
> > > >>>>>>>>
> > > >>>>>>>> Let's say we're joining a left table (Integer K: Letter FK,
> > (other
> > > >>>>>>> data))
> > > >>>>>>>> to a right table (Letter K: (some data)).
> > > >>>>>>>>
> > > >>>>>>>> Left table:
> > > >>>>>>>> 1: (A, xyz)
> > > >>>>>>>> 2: (B, asd)
> > > >>>>>>>>
> > > >>>>>>>> Right table:
> > > >>>>>>>> A: EntityA
> > > >>>>>>>> B: EntityB
> > > >>>>>>>>
> > > >>>>>>>> We could do a rekey as you proposed with a combined key, but
> not
> > > >>>>>>>> propagating the value at all..
> > > >>>>>>>> Rekey table:
> > > >>>>>>>> A-1: (dummy value)
> > > >>>>>>>> B-2: (dummy value)
> > > >>>>>>>>
> > > >>>>>>>> Which we then join with the right table to produce:
> > > >>>>>>>> A-1: EntityA
> > > >>>>>>>> B-2: EntityB
> > > >>>>>>>>
> > > >>>>>>>> Which gets rekeyed back:
> > > >>>>>>>> 1: A, EntityA
> > > >>>>>>>> 2: B, EntityB
> > > >>>>>>>>
> > > >>>>>>>> And finally we do the actual join:
> > > >>>>>>>> Result table:
> > > >>>>>>>> 1: ((A, xyz), EntityA)
> > > >>>>>>>> 2: ((B, asd), EntityB)
> > > >>>>>>>>
> > > >>>>>>>> The thing is that in that last join, we have the opportunity
> to
> > > >>>>> compare
> > > >>>>>>> the
> > > >>>>>>>> current FK in the left table with the incoming PK of the right
> > > >>>>> table. If
> > > >>>>>>>> they don't match, we just drop the event, since it must be
> > > outdated.
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets
> > updated
> > > >> to
> > > >>>>>>> (1:
> > > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether the
> final
> > > >>>>> state
> > > >>>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm
> > above,
> > > >>>>> you
> > > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz),
> > (B,
> > > >>>>>>>> EntityB)). It seems like this does give you enough information
> > to
> > > >>>>> make
> > > >>>>>>> the
> > > >>>>>>>> right choice, regardless of disordering.
> > > >>>>>>>
> > > >>>>>>> Will check Adams patch, but this should work. As mentioned
> often
> > I
> > > am
> > > >>>>>>> not convinced on partitioning back for the user automatically.
> I
> > > >> think
> > > >>>>>>> this is the real performance eater ;)
> > > >>>>>>>
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> 7. Last thought... I'm a little concerned about the
> performance
> > of
> > > >>>>> the
> > > >>>>>>>> range scans when records change in the right table. You've
> said
> > > that
> > > >>>>>>> you've
> > > >>>>>>>> been using the algorithm you presented in production for a
> > while.
> > > >> Can
> > > >>>>>>> you
> > > >>>>>>>> give us a sense of the performance characteristics you've
> > > observed?
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>> Make it work, make it fast, make it beautiful. The topmost
> thing
> > > here
> > > >>>>> is
> > > >>>>>>> / was correctness. In practice I do not measure the performance
> > of
> > > >> the
> > > >>>>>>> range scan. Usual cases I run this with is emitting 500k - 1kk
> > rows
> > > >>>>>>> on a left hand side change. The range scan is just the work you
> > > gotta
> > > >>>>>>> do, also when you pack your data into different formats,
> usually
> > > the
> > > >>>>>>> rocks performance is very tight to the size of the data and we
> > > can't
> > > >>>>>>> really change that. It is more important for users to prevent
> > > useless
> > > >>>>>>> updates to begin with. My left hand side is guarded to drop
> > changes
> > > >>>>> that
> > > >>>>>>> are not going to change my join output.
> > > >>>>>>>
> > > >>>>>>> usually it's:
> > > >>>>>>>
> > > >>>>>>> drop unused fields and then don't forward if old.equals(new)
> > > >>>>>>>
> > > >>>>>>> regarding to the performance of creating an iterator for
> smaller
> > > >>>>>>> fanouts, users can still just do a group by first then anyways.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>> I could only think of one alternative, but I'm not sure if
> it's
> > > >>>>> better
> > > >>>>>>> or
> > > >>>>>>>> worse... If the first re-key only needs to preserve the
> original
> > > >> key,
> > > >>>>>>> as I
> > > >>>>>>>> proposed in #6, then we could store a vector of keys in the
> > value:
> > > >>>>>>>>
> > > >>>>>>>> Left table:
> > > >>>>>>>> 1: A,...
> > > >>>>>>>> 2: B,...
> > > >>>>>>>> 3: A,...
> > > >>>>>>>>
> > > >>>>>>>> Gets re-keyed:
> > > >>>>>>>> A: [1, 3]
> > > >>>>>>>> B: [2]
> > > >>>>>>>>
> > > >>>>>>>> Then, the rhs part of the join would only need a regular
> > > single-key
> > > >>>>>>> lookup.
> > > >>>>>>>> Of course we have to deal with the problem of large values, as
> > > >>>>> there's
> > > >>>>>>> no
> > > >>>>>>>> bound on the number of lhs records that can reference rhs
> > records.
> > > >>>>>>> Offhand,
> > > >>>>>>>> I'd say we could page the values, so when one row is past the
> > > >>>>>>> threshold, we
> > > >>>>>>>> append the key for the next page. Then in most cases, it would
> > be
> > > a
> > > >>>>>>> single
> > > >>>>>>>> key lookup, but for large fan-out updates, it would be one per
> > > (max
> > > >>>>>>> value
> > > >>>>>>>> size)/(avg lhs key size).
> > > >>>>>>>>
> > > >>>>>>>> This seems more complex, though... Plus, I think there's some
> > > extra
> > > >>>>>>>> tracking we'd need to do to know when to emit a retraction.
> For
> > > >>>>> example,
> > > >>>>>>>> when record 1 is deleted, the re-key table would just have (A:
> > > [3]).
> > > >>>>>>> Some
> > > >>>>>>>> kind of tombstone is needed so that the join result for 1 can
> > also
> > > >> be
> > > >>>>>>>> retracted.
> > > >>>>>>>>
> > > >>>>>>>> That's all!
> > > >>>>>>>>
> > > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP.
> > Sorry
> > > >> the
> > > >>>>>>>> discussion has been slow.
> > > >>>>>>>> -John
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > > >>>>> Jan.Filipiak@trivago.com>
> > > >>>>>>>> wrote:
> > > >>>>>>>>
> > > >>>>>>>>> Id say you can just call the vote.
> > > >>>>>>>>>
> > > >>>>>>>>> that happens all the time, and if something comes up, it just
> > > goes
> > > >>>>> back
> > > >>>>>>>>> to discuss.
> > > >>>>>>>>>
> > > >>>>>>>>> would not expect to much attention with another another email
> > in
> > > >>>>> this
> > > >>>>>>>>> thread.
> > > >>>>>>>>>
> > > >>>>>>>>> best Jan
> > > >>>>>>>>>
> > > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > > >>>>>>>>>> Hello Contributors
> > > >>>>>>>>>>
> > > >>>>>>>>>> I know that 2.1 is about to be released, but I do need to
> bump
> > > >>>>> this to
> > > >>>>>>>>> keep
> > > >>>>>>>>>> visibility up. I am still intending to push this through
> once
> > > >>>>>>> contributor
> > > >>>>>>>>>> feedback is given.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Main points that need addressing:
> > > >>>>>>>>>> 1) Any way (or benefit) in structuring the current singular
> > > graph
> > > >>>>> node
> > > >>>>>>>>> into
> > > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I
> > am
> > > a
> > > >>>>> bit
> > > >>>>>>>>> fuzzy
> > > >>>>>>>>>> on how the optimizations are supposed to work, so I would
> > > >>>>> appreciate
> > > >>>>>>> any
> > > >>>>>>>>>> help on this aspect.
> > > >>>>>>>>>>
> > > >>>>>>>>>> 2) Overall strategy for joining + resolving. This thread has
> > > much
> > > >>>>>>>>> discourse
> > > >>>>>>>>>> between Jan and I between the current highwater mark
> proposal
> > > and
> > > >> a
> > > >>>>>>>>> groupBy
> > > >>>>>>>>>> + reduce proposal. I am of the opinion that we need to
> > strictly
> > > >>>>> handle
> > > >>>>>>>>> any
> > > >>>>>>>>>> chance of out-of-order data and leave none of it up to the
> > > >>>>> consumer.
> > > >>>>>>> Any
> > > >>>>>>>>>> comments or suggestions here would also help.
> > > >>>>>>>>>>
> > > >>>>>>>>>> 3) Anything else that you see that would prevent this from
> > > moving
> > > >>>>> to a
> > > >>>>>>>>> vote?
> > > >>>>>>>>>>
> > > >>>>>>>>>> Thanks
> > > >>>>>>>>>>
> > > >>>>>>>>>> Adam
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > > >>>>>>>>> adam.bellemare@gmail.com>
> > > >>>>>>>>>> wrote:
> > > >>>>>>>>>>
> > > >>>>>>>>>>> Hi Jan
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> > > >>>>> Stores.persistentWindowStore,
> > > >>>>>>> you
> > > >>>>>>>>>>> actually only need to specify the amount of segments you
> want
> > > and
> > > >>>>> how
> > > >>>>>>>>> large
> > > >>>>>>>>>>> they are. To the best of my understanding, what happens is
> > that
> > > >>>>> the
> > > >>>>>>>>>>> segments are automatically rolled over as new data with new
> > > >>>>>>> timestamps
> > > >>>>>>>>> are
> > > >>>>>>>>>>> created. We use this exact functionality in some of the
> work
> > > done
> > > >>>>>>>>>>> internally at my company. For reference, this is the
> hopping
> > > >>>>> windowed
> > > >>>>>>>>> store.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>
> > > >>>>>
> > > >>
> > >
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> In the code that I have provided, there are going to be two
> > 24h
> > > >>>>>>>>> segments.
> > > >>>>>>>>>>> When a record is put into the windowStore, it will be
> > inserted
> > > at
> > > >>>>>>> time
> > > >>>>>>>>> T in
> > > >>>>>>>>>>> both segments. The two segments will always overlap by 12h.
> > As
> > > >>>>> time
> > > >>>>>>>>> goes on
> > > >>>>>>>>>>> and new records are added (say at time T+12h+), the oldest
> > > >> segment
> > > >>>>>>> will
> > > >>>>>>>>> be
> > > >>>>>>>>>>> automatically deleted and a new segment created. The
> records
> > > are
> > > >>>>> by
> > > >>>>>>>>> default
> > > >>>>>>>>>>> inserted with the context.timestamp(), such that it is the
> > > record
> > > >>>>>>> time,
> > > >>>>>>>>> not
> > > >>>>>>>>>>> the clock time, which is used.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> To the best of my understanding, the timestamps are
> retained
> > > when
> > > >>>>>>>>>>> restoring from the changelog.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
> > > >>>>>>> segment-level,
> > > >>>>>>>>>>> instead of at an individual record level.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > > >>>>>>> Jan.Filipiak@trivago.com>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Will that work? I expected it to blow up with
> > > ClassCastException
> > > >>>>> or
> > > >>>>>>>>>>>> similar.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> You either would have to specify the window you fetch/put
> or
> > > >>>>> iterate
> > > >>>>>>>>>>>> across all windows the key was found in right?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> I just hope the window-store doesn't check stream-time
> under
> > > the
> > > >>>>>>> hoods
> > > >>>>>>>>>>>> that would be a questionable interface.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> If it does: did you see my comment on checking all the
> > windows
> > > >>>>>>> earlier?
> > > >>>>>>>>>>>> that would be needed to actually give reasonable time
> > > gurantees.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Best
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > > >>>>>>>>>>>>> Hi Jan
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the
> > > state
> > > >>>>>>> store,
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>> the ProcessorSupplier.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>> Adam
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> @Guozhang
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks for the information. This is indeed something
> that
> > > >>>>> will be
> > > >>>>>>>>>>>>>>> extremely
> > > >>>>>>>>>>>>>>> useful for this KIP.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> @Jan
> > > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I will
> not
> > > be
> > > >>>>>>> moving
> > > >>>>>>>>>>>> ahead
> > > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution
> > as
> > > >> you
> > > >>>>>>>>>>>> propose.
> > > >>>>>>>>>>>>>>> That being said, if you wish to implement it yourself
> off
> > > of
> > > >>>>> my
> > > >>>>>>>>>>>> current PR
> > > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I would be
> > more
> > > >>>>> than
> > > >>>>>>>>>>>> happy to
> > > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it stands
> > right
> > > >>>>> now,
> > > >>>>>>> I do
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>> really have more time to invest into alternatives
> without
> > > >>>>> there
> > > >>>>>>>>> being
> > > >>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>> strong indication from the binding voters which they
> > would
> > > >>>>>>> prefer.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on
> the
> > > >>>>> streams
> > > >>>>>>>>> DSL
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>> some time already, otherwise I would have pulled this
> KIP
> > > >>>>> through
> > > >>>>>>>>>>>> already.
> > > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed
> state
> > > >>>>> store
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I will
> come
> > > >> back
> > > >>>>>>> for
> > > >>>>>>>>>>>> final
> > > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the
> > > binding
> > > >>>>>>> voters
> > > >>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
> > > >>>>> according
> > > >>>>>>> to
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> latest plan:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store.
> > > This
> > > >>>>>>> could
> > > >>>>>>>>> be
> > > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
> > > >>>>> completed.
> > > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Adam
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already
> > > updated
> > > >>>>> in
> > > >>>>>>> the
> > > >>>>>>>>>>>> PR?
> > > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
> > something?
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > > >>>>>>> wangguoz@gmail.com>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the
> wrong
> > > >> link,
> > > >>>>>>> as it
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of
> > KIP-258
> > > >>>>> we do
> > > >>>>>>>>>>>> want to
> > > >>>>>>>>>>>>>>>> have "handling out-of-order data for source KTable"
> such
> > > >> that
> > > >>>>>>>>>>>> instead of
> > > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized store,
> > i.e.
> > > >>>>>>> following
> > > >>>>>>>>>>>>>>>> offset
> > > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older than
> the
> > > >>>>> current
> > > >>>>>>>>>>>> key's
> > > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Guozhang
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> > > >>>>>>>>> wangguoz@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hello Adam,
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step
> > > (i.e.
> > > >>>>> the
> > > >>>>>>>>> high
> > > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a
> > window
> > > >>>>>>> store),
> > > >>>>>>>>> I
> > > >>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >>>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value
> store
> > > >>>>> (i.e.
> > > >>>>>>> only
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
> > > >>>>> described
> > > >>>>>>> in
> > > >>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is
> > > that
> > > >>>>> we
> > > >>>>>>> can
> > > >>>>>>>>>>>> then
> > > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
> > timestamp
> > > is
> > > >>>>>>>>> smaller
> > > >>>>>>>>>>>> than
> > > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I think it
> > is
> > > >>>>> very
> > > >>>>>>>>>>>> similar to
> > > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
> > filtering,
> > > >>>>> while
> > > >>>>>>>>> you
> > > >>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the joining
> > > >> records
> > > >>>>>>> are
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> correctly
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> inherited though the whole topology to the final
> stage.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
> > > >>>>>>> non-windowed
> > > >>>>>>>>>>>> KTables
> > > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really have
> a
> > > good
> > > >>>>>>>>> support
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>> their joins anyways (
> > > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > > >>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
> > > >>>>> non-key
> > > >>>>>>>>> joins
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Guozhang
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Hi Guozhang
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
> > > >> endlessly
> > > >>>>>>> based
> > > >>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of
> (<this
> > > >>>>> table
> > > >>>>>>>>>>>> primary
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> key>,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
> > > >>>>>>> differentiate
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> between
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal
> would
> > > be
> > > >>>>> to
> > > >>>>>>>>>>>> replace
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would
> > > allow
> > > >>>>> the
> > > >>>>>>>>> same
> > > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This
> > should
> > > >>>>> allow
> > > >>>>>>> for
> > > >>>>>>>>>>>> all
> > > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
> > > >>>>>>> customizable
> > > >>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just
> > 10
> > > >>>>>>> minutes
> > > >>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> window,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the
> trick
> > > >> here.
> > > >>>>>>> Even
> > > >>>>>>>>>>>> if I
> > > >>>>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
> > > >>>>> optional
> > > >>>>>>>>>>>> since I
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a
> little
> > > bit
> > > >>>>>>>>>>>> sceptical
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one could
> > run
> > > >>>>> into
> > > >>>>>>>>>>>> problems
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> the rapid change happens near a window border. I will
> > > check
> > > >>>>> you
> > > >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we
> could
> > > >>>>> still
> > > >>>>>>>>> check
> > > >>>>>>>>>>>>>>>>>> _all_
> > > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance impact I
> > > >> guess.
> > > >>>>>>> Will
> > > >>>>>>>>>>>> let
> > > >>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
> > > >>>>> wouldn't
> > > >>>>>>> not
> > > >>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> > > >>>>>>>>> timestamp(B).
> > > >>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> we can't expect that.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> @Jan
> > > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks
> for
> > > the
> > > >>>>>>>>>>>> diagram, it
> > > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not have
> > the
> > > >>>>>>> original
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> primary
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> key available, and I can see that if it was available
> > > then
> > > >>>>> you
> > > >>>>>>>>>>>> would be
> > > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That
> > being
> > > >>>>> said,
> > > >>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> encourage
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity
> > for
> > > >>>>>>> everyone
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> else.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard
> work.
> > > But
> > > >>>>> I
> > > >>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original
> > > >> primary
> > > >>>>>>> key,
> > > >>>>>>>>> We
> > > >>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
> > > >> basically
> > > >>>>>>> not
> > > >>>>>>>>>>>> using
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
> > > >>>>> original
> > > >>>>>>> DSL
> > > >>>>>>>>>>>> its
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my
> > end.
> > > >>>>> Will
> > > >>>>>>>>>>>> finish
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay
> > > >> inside
> > > >>>>>>> the
> > > >>>>>>>>>>>> State
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
> > > >>>>> propagated?
> > > >>>>>>>>> Isn't
> > > >>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state
> store?
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is
> > gonna
> > > >>>>>>> return
> > > >>>>>>>>>>>> `null`
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is
> > going
> > > to
> > > >>>>> be
> > > >>>>>>> a
> > > >>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store
> > > directly
> > > >>>>> for
> > > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a
> regular
> > > >>>>> store,
> > > >>>>>>>>>>>> satisfying
> > > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
> > > >>>>> Windowed
> > > >>>>>>>>>>>> store is
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull
> operation
> > > we
> > > >>>>>>> would
> > > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the
> > > window
> > > >>>>>>> store
> > > >>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> the values then.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group
> by
> > > >>>>> before
> > > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i think
> it
> > > >> would
> > > >>>>>>> help
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> users
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the
> original
> > > >>>>> primary
> > > >>>>>>>>> key
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> issue I
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> understand that we do not have a solid foundation to
> > > build
> > > >>>>> on.
> > > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
> > > >>>>>>> unfortunate. I
> > > >>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not
> think
> > > its
> > > >>>>> a
> > > >>>>>>> good
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> decision.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Thanks
> > > >>>>>>>>>>>>>>>>>>> Adam
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> > > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > > >>>>>>> dumbreprajakta311@gmail.com
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan
> > Filipiak
> > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> > > >>>>>>>>>>>>>>>>>>>           >
> > > >>>>>>>>>>>>>>>>>>>           > give me some time, will make such a
> > chart.
> > > >> last
> > > >>>>>>> time i
> > > >>>>>>>>>>>> didn't
> > > >>>>>>>>>>>>>>>>>>>           get along
> > > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your
> > charts.
> > > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> > > >>>>>>>>>>>>>>>>>>>           >
> > > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare
> > wrote:
> > > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of what I
> > > >> attempted
> > > >>>>> on
> > > >>>>>>> the
> > > >>>>>>>>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           >
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > > >>>>>>>>>>>>>>>>>>>           <
> > > >>>>>>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the start of
> > my
> > > own
> > > >>>>>>>>>>>> implementation
> > > >>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>           this
> > > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not get
> it
> > to
> > > >>>>> work I
> > > >>>>>>> have
> > > >>>>>>>>>>>> since
> > > >>>>>>>>>>>>>>>>>>>           discarded the
> > > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if you
> > wish
> > > to
> > > >>>>>>> continue
> > > >>>>>>>>>>>> pursuing
> > > >>>>>>>>>>>>>>>>>>>           for your
> > > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you
> please
> > > >>>>> create a
> > > >>>>>>>>>>>> diagram on
> > > >>>>>>>>>>>>>>>>>>>           the KIP
> > > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your solution.
> > > Please
> > > >>>>> feel
> > > >>>>>>> free
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>           the image I
> > > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting point. I am
> > > having
> > > >>>>>>> trouble
> > > >>>>>>>>>>>>>>>>>>>           understanding your
> > > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a
> > carefully
> > > >>>>>>> constructed
> > > >>>>>>>>>>>> diagram
> > > >>>>>>>>>>>>>>>>>>>           will clear
> > > >>>>>>>>>>>>>>>>>>>           > up
> > > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings. Alternately,
> > > please
> > > >>>>> post a
> > > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> > > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess at
> what
> > > you
> > > >>>>>>> mean, and
> > > >>>>>>>>>>>> since I
> > > >>>>>>>>>>>>>>>>>>>           value my
> > > >>>>>>>>>>>>>>>>>>>           > own
> > > >>>>>>>>>>>>>>>>>>>           > > time as much as you value yours, I
> > > believe
> > > >> it
> > > >>>>> is
> > > >>>>>>> your
> > > >>>>>>>>>>>>>>>>>>>           responsibility to
> > > >>>>>>>>>>>>>>>>>>>           > > provide an implementation instead of
> me
> > > >>>>> trying to
> > > >>>>>>>>> guess.
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > > Adam
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM, Jan
> > > Filipiak
> > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > > wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >
> > > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing interested.
> > kafka
> > > >>>>>>> streams at
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>           point supports
> > > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as both
> > > streams
> > > >>>>> have
> > > >>>>>>> the
> > > >>>>>>>>>>>> same
> > > >>>>>>>>>>>>>>>>>>> key.
> > > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a
> join
> > > >> where a
> > > >>>>>>> KTable
> > > >>>>>>>>>>>> and a
> > > >>>>>>>>>>>>>>>>>>>           KTable can
> > > >>>>>>>>>>>>>>>>>>>           > have
> > > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship (1:n).
> We
> > > >> exploit
> > > >>>>>>> that
> > > >>>>>>>>>>>> rocksdb
> > > >>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted (At
> > least
> > > >>>>>>> exposes an
> > > >>>>>>>>>>>> API to
> > > >>>>>>>>>>>>>>>>>>>           access the
> > > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted fashion).
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats are
> well
> > > >>>>>>> understood
> > > >>>>>>>>> now
> > > >>>>>>>>>>>> and we
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > basically
> > > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API Design (
> > when
> > > >> Adam
> > > >>>>>>> sees
> > > >>>>>>>>> my
> > > >>>>>>>>>>>> newest
> > > >>>>>>>>>>>>>>>>>>>           message).
> > > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of
> > loosing
> > > >>>>> those
> > > >>>>>>> kinda
> > > >>>>>>>>>>>>>>>>>>>           arguments within
> > > >>>>>>>>>>>>>>>>>>>           > the
> > > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have no clue
> > > why.
> > > >> So
> > > >>>>> I
> > > >>>>>>>>>>>> literally
> > > >>>>>>>>>>>>>>>>>>>           can't wait for
> > > >>>>>>>>>>>>>>>>>>>           > you
> > > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and
> give
> > > you
> > > >>>>>>> opinion on
> > > >>>>>>>>>>>> how we
> > > >>>>>>>>>>>>>>>>>>>           should
> > > >>>>>>>>>>>>>>>>>>>           > design
> > > >>>>>>>>>>>>>>>>>>>           > >> the return type of the oneToManyJoin
> > and
> > > >> how
> > > >>>>>>> many
> > > >>>>>>>>>>>> power we
> > > >>>>>>>>>>>>>>>>>>>           want to give
> > > >>>>>>>>>>>>>>>>>>>           > to
> > > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where
> > > simplicity
> > > >>>>> isn't
> > > >>>>>>>>>>>> really that
> > > >>>>>>>>>>>>>>>>>>>           as users
> > > >>>>>>>>>>>>>>>>>>>           > still
> > > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on the
> > > >> discussion
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan
> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>
> > > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I found
> > this
> > > >>>>> subject
> > > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> > > >>>>>>>>>>>>>>>>>>>           > like
> > > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a join
> > > table of
> > > >>>>> two
> > > >>>>>>>>>>>> streams? Is
> > > >>>>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>           > somewhere
> > > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original requirement
> or
> > > >>>>> proposal?
> > > >>>>>>>>>>>>>>>>>>>           > >>>
> > > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan
> > > Filipiak
> > > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam
> Bellemare
> > > >> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a
> > > Windowed
> > > >>>>> Store
> > > >>>>>>> to
> > > >>>>>>>>>>>> store the
> > > >>>>>>>>>>>>>>>>>>>           highwater
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this should
> work
> > > >> fine,
> > > >>>>>>> with
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> caveat
> > > >>>>>>>>>>>>>>>>>>>           being that
> > > >>>>>>>>>>>>>>>>>>>           > it
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order arrival
> > > for up
> > > >>>>> to
> > > >>>>>>> the
> > > >>>>>>>>>>>> size of
> > > >>>>>>>>>>>>>>>>>>>           the window
> > > >>>>>>>>>>>>>>>>>>>           > (ie:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would remove
> > the
> > > >>>>>>> possibility
> > > >>>>>>>>>>>> of it
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> being
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > unbounded
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's
> suggestion, I
> > > >>>>> believe
> > > >>>>>>> this
> > > >>>>>>>>> is
> > > >>>>>>>>>>>> where
> > > >>>>>>>>>>>>>>>>>>>           we will
> > > >>>>>>>>>>>>>>>>>>>           > have
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement. While I
> do
> > > not
> > > >>>>>>> disagree
> > > >>>>>>>>>>>> with your
> > > >>>>>>>>>>>>>>>>>>>           statement
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional
> joins
> > > done
> > > >>>>> in a
> > > >>>>>>>>>>>> real-world
> > > >>>>>>>>>>>>>>>>>>>           workflow, I
> > > >>>>>>>>>>>>>>>>>>>           > do
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively deal
> > > with
> > > >>>>>>>>> out-of-order
> > > >>>>>>>>>>>>>>>>>>> arrival
> > > >>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent joins. I
> > have
> > > >>>>>>> attempted
> > > >>>>>>>>> what
> > > >>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>           think you have
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a high-water,
> > using
> > > >>>>>>> groupBy and
> > > >>>>>>>>>>>> reduce)
> > > >>>>>>>>>>>>>>>>>>>           and found
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too
> > quickly,
> > > or
> > > >>>>> the
> > > >>>>>>> load
> > > >>>>>>>>> on
> > > >>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>           stream thread
> > > >>>>>>>>>>>>>>>>>>>           > is
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages will
> > arrive
> > > >>>>>>>>> out-of-order
> > > >>>>>>>>>>>> and be
> > > >>>>>>>>>>>>>>>>>>>           incorrectly
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> > intermediate
> > > >>>>> event
> > > >>>>>>> is
> > > >>>>>>>>>>>>>>>>>>> represented
> > > >>>>>>>>>>>>>>>>>>>           as the
> > > >>>>>>>>>>>>>>>>>>>           > final
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on your
> > > groupBy
> > > >>>>>>>>>>>> implementation.
> > > >>>>>>>>>>>>>>>>>>>           There must be
> > > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> > > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it is, I
> > > would
> > > >>>>> just
> > > >>>>>>> like
> > > >>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> > > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
> > > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I would
> > > like
> > > >> to
> > > >>>>>>> clarify
> > > >>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>           we draw a
> > > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the scattered
> > events
> > > >>>>> back to
> > > >>>>>>>>> their
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I know
> how
> > > to
> > > >>>>>>>>> conclusively
> > > >>>>>>>>>>>> deal
> > > >>>>>>>>>>>>>>>>>>>           with
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a given
> time
> > > >> frame,
> > > >>>>>>> and to
> > > >>>>>>>>>>>> ensure
> > > >>>>>>>>>>>>>>>>>>>           that the
> > > >>>>>>>>>>>>>>>>>>>           > data
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with the
> > input
> > > >>>>> events.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to share
> that
> > > >>>>>>> illustrates
> > > >>>>>>>>> your
> > > >>>>>>>>>>>>>>>>>>>           approach, I
> > > >>>>>>>>>>>>>>>>>>>           > would
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would remove
> > any
> > > >>>>>>>>>>>> misunderstandings
> > > >>>>>>>>>>>>>>>>>>>           that I may
> > > >>>>>>>>>>>>>>>>>>>           > have.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for my
> > code.
> > > I
> > > >>>>> don't
> > > >>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>           something easily
> > > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated with
> > > >>>>> OO-patterns.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> > > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey, V
> > > value, T
> > > >>>>>>>>> aggregate)
> > > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> > currentStateAsMap =
> > > >>>>>>>>>>>> asMap(aggregate);
> > > >>>>>>>>>>>>>>>>>>> <<
> > > >>>>>>>>>>>>>>>>>>>           imaginary
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> > > >>>>> mapper.apply(value);
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the place
> > > where
> > > >>>>> people
> > > >>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>           gonna have
> > > >>>>>>>>>>>>>>>>>>>           > issues
> > > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't do
> it.
> > > we
> > > >>>>> would
> > > >>>>>>> need
> > > >>>>>>>>>>>> to find
> > > >>>>>>>>>>>>>>>>>>>           a solution
> > > >>>>>>>>>>>>>>>>>>>           > here.
> > > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate the
> > > field in
> > > >>>>> the
> > > >>>>>>>>>>>> joiner, so
> > > >>>>>>>>>>>>>>>>>>>           that we can
> > > >>>>>>>>>>>>>>>>>>>           > pick
> > > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate. Probably
> you
> > > have
> > > >>>>> not
> > > >>>>>>>>> thought
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>           this in your
> > > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open to
> > > find a
> > > >>>>>>> generic
> > > >>>>>>>>>>>> solution
> > > >>>>>>>>>>>>>>>>>>>           here. In my
> > > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken in
> > > >>>>>>>>> KTableImpl.GroupBy
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>           looses
> > > >>>>>>>>>>>>>>>>>>>           > the keys
> > > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the aggregate
> > key.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted it
> away
> > > back
> > > >>>>>>> then way
> > > >>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>> i
> > > >>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>           > thinking
> > > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is why I
> > > didn't
> > > >>>>>>> realize
> > > >>>>>>>>> its
> > > >>>>>>>>>>>>>>>>>>>           significance here.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > currentStateAsMap.put(mapper.apply(m),
> > > >> m);
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > >>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(toModifyKey,
> > > >> value);
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > currentStateAsMap.remove(toModifyKey);
> > > >>>>>>>>>>>>>>>>>>>           > >>>> if(currentStateAsMap.isEmpty()){
> > > >>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
> > > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> > > >>>>>>> asAggregateType(currentStateAsMap)
> > > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35 PM,
> Jan
> > > >>>>> Filipiak
> > > >>>>>>> <
> > > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
> > > >>>>>>>>> Jan.Filipiak@trivago.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing Matthias
> > to
> > > >>>>> speed!
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I think
> > > >> re-keying
> > > >>>>>>> back
> > > >>>>>>>>>>>> should be
> > > >>>>>>>>>>>>>>>>>>>           optional at
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> > > KScatteredTable
> > > >>>>> with
> > > >>>>>>>>>>>> reshuffle()
> > > >>>>>>>>>>>>>>>>>>>           returning
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined> to
> make
> > > the
> > > >>>>>>> backwards
> > > >>>>>>>>>>>>>>>>>>>           repartitioning
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of
> doing
> > > the
> > > >>>>> out
> > > >>>>>>> of
> > > >>>>>>>>> order
> > > >>>>>>>>>>>>>>>>>>>           processing using
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark
> > tracking.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded growth is
> > > just
> > > >>>>> scary
> > > >>>>>>> + It
> > > >>>>>>>>>>>> saves
> > > >>>>>>>>>>>>>>>>>>> us
> > > >>>>>>>>>>>>>>>>>>>           the header
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of
> always
> > > >>>>>>> repartitioning
> > > >>>>>>>>>>>> back is
> > > >>>>>>>>>>>>>>>>>>>           just not so
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has been
> > done
> > > >>>>> before
> > > >>>>>>> we
> > > >>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>           back and
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards is
> > > really
> > > >>>>>>> common.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
> > Bellemare
> > > >>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your feedback, I
> do
> > > >>>>>>> appreciate
> > > >>>>>>>>> it!
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be
> > > possible,
> > > >> it
> > > >>>>>>> would
> > > >>>>>>>>>>>> require
> > > >>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>           > deserialize
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > > runtime
> > > >>>>>>> overhead.
> > > >>>>>>>>> I
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > >>>>>>>>>>>>>>>>>>>           > no
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> > > >>>>> overhead.
> > > >>>>>>> If
> > > >>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> becomes a
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > problem in
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> > name
> > > >>>>> spacing
> > > >>>>>>>>> later
> > > >>>>>>>>>>>> on.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with using a
> > > >> reserved
> > > >>>>>>> string
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>           document it.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the
> design
> > it
> > > >> the
> > > >>>>>>> type of
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > >>>>>>>>>>>>>>>>>>>           > If
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
> > correctly,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have
> table1
> > > and
> > > >>>>> table2
> > > >>>>>>>>>>>> swapped.
> > > >>>>>>>>>>>>>>>>>>>           Here is how it
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records that
> > > contain
> > > >>>>> the
> > > >>>>>>>>>>>> foreign key
> > > >>>>>>>>>>>>>>>>>>>           within their
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> > > <a,(fk=A,bar=1)>,
> > > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>,
> <B,Y>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is required
> to
> > > >> extract
> > > >>>>>>> the
> > > >>>>>>>>>>>> foreign
> > > >>>>>>>>>>>>>>>>>>> key.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper: (
> > value
> > > =>
> > > >>>>>>> value.fk
> > > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to each
> > > element
> > > >> in
> > > >>>>>>>>> table1,
> > > >>>>>>>>>>>> and a
> > > >>>>>>>>>>>>>>>>>>>           new combined
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> > (fk=A,bar=1)>,
> > > >>>>> <A-b,
> > > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > > >>>>>>>>>>>>>>>>>>>           <B-c,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> > > >> copartitioned
> > > >>>>>>> with
> > > >>>>>>>>>>>> table2:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with Partition
> > 0:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
> > > >>>>> (fk=A,bar=1)>,
> > > >>>>>>> <A-b,
> > > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with Partition
> > 1:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
> > > >> (fk=B,bar=3)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be
> joined
> > > >>>>> together
> > > >>>>>>>>> locally
> > > >>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>>>           applying the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design and
> > my
> > > >>>>> design
> > > >>>>>>>>>>>> deviate. My
> > > >>>>>>>>>>>>>>>>>>>           design goes
> > > >>>>>>>>>>>>>>>>>>>           > on
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data post-join
> > and
> > > >>>>> resolve
> > > >>>>>>>>>>>> out-of-order
> > > >>>>>>>>>>>>>>>>>>>           arrival of
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data
> keyed
> > > just
> > > >>>>> the
> > > >>>>>>>>>>>> original key.
> > > >>>>>>>>>>>>>>>>>>>           I do not
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the
> > internals
> > > >>>>>>> outside of
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make for
> > larger
> > > >>>>>>> footprint,
> > > >>>>>>>>>>>> but it
> > > >>>>>>>>>>>>>>>>>>>           removes all
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order
> > arrivals
> > > >> and
> > > >>>>>>>>> handling
> > > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> > > >>>>>>>>>>>>>>>>>>>           > the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this makes
> > the
> > > >>>>>>> function
> > > >>>>>>>>> much
> > > >>>>>>>>>>>>>>>>>>> easier
> > > >>>>>>>>>>>>>>>>>>>           to use.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps
> resolve
> > > your
> > > >>>>>>>>> questions,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>           please feel
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your mind.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
> > > >>>>> Matthias J.
> > > >>>>>>>>> Sax <
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io <mailto:
> > > >>>>>>>>>>>> matthias@confluent.io>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on this
> > > >> thread. I
> > > >>>>>>> did
> > > >>>>>>>>> not
> > > >>>>>>>>>>>> read
> > > >>>>>>>>>>>>>>>>>>>           everything so
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple of
> > > initial
> > > >>>>>>> thoughts:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is a
> > > >> fundamental
> > > >>>>>>>>>>>> difference
> > > >>>>>>>>>>>>>>>>>>>           between header
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258. For
> 258,
> > > we
> > > >> add
> > > >>>>>>>>> headers
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams and
> > > nobody
> > > >>>>>>> else is
> > > >>>>>>>>>>>> supposed
> > > >>>>>>>>>>>>>>>>>>>           to write
> > > >>>>>>>>>>>>>>>>>>>           > into
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user header
> > are
> > > >>>>> written
> > > >>>>>>> into
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           changelog topic
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not conflicts.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see a
> big
> > > issue
> > > >>>>> with
> > > >>>>>>>>> using
> > > >>>>>>>>>>>>>>>>>>>           headers within
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it, we
> > can
> > > >> have
> > > >>>>>>> some
> > > >>>>>>>>>>>> "reserved"
> > > >>>>>>>>>>>>>>>>>>>           header keys
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to use
> > when
> > > >>>>>>> processing
> > > >>>>>>>>>>>> data with
> > > >>>>>>>>>>>>>>>>>>>           Kafka
> > > >>>>>>>>>>>>>>>>>>>           > Streams.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe way to
> > > avoid
> > > >>>>>>>>> conflicts,
> > > >>>>>>>>>>>> since
> > > >>>>>>>>>>>>>>>>>>> these
> > > >>>>>>>>>>>>>>>>>>>           > headers
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal
> topics
> > (I
> > > >>>>> think):
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and changelog
> > > topics,
> > > >> we
> > > >>>>> can
> > > >>>>>>>>>>>> namespace
> > > >>>>>>>>>>>>>>>>>>>           all headers:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers are
> > > >> namespaced
> > > >>>>> as
> > > >>>>>>>>>>>> "external."
> > > >>>>>>>>>>>>>>>>>>> +
> > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> > > namespaced as
> > > >>>>>>>>>>>> "internal." +
> > > >>>>>>>>>>>>>>>>>>>           headerKey
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would be
> > > >> possible,
> > > >>>>> it
> > > >>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>> require
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > > runtime
> > > >>>>>>> overhead.
> > > >>>>>>>>> I
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>           suggest to
> > > >>>>>>>>>>>>>>>>>>>           > no
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> > > >>>>> overhead.
> > > >>>>>>> If
> > > >>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> becomes a
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > problem in
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> > name
> > > >>>>> spacing
> > > >>>>>>>>> later
> > > >>>>>>>>>>>> on.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the
> design
> > > it
> > > >> the
> > > >>>>>>> type
> > > >>>>>>>>> of
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           result KTable:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
> > correctly,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable =
> > > >>>>>>>>>>>> table1.join(table2,...);
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the `joinedTable`
> > has
> > > >> the
> > > >>>>>>> same
> > > >>>>>>>>> key
> > > >>>>>>>>>>>> as the
> > > >>>>>>>>>>>>>>>>>>>           left input
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work
> because
> > > if
> > > >>>>> table2
> > > >>>>>>>>>>>> contains
> > > >>>>>>>>>>>>>>>>>>>           multiple rows
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in table1
> > > (what is
> > > >>>>> the
> > > >>>>>>> main
> > > >>>>>>>>>>>> purpose
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>           > foreign
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table would
> > only
> > > >>>>>>> contain a
> > > >>>>>>>>>>>> single
> > > >>>>>>>>>>>>>>>>>>>           join result,
> > > >>>>>>>>>>>>>>>>>>>           > but
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream:
> <a,(A,1)>,
> > > >>>>> <b,(A,2)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a foreign
> > key
> > > to
> > > >>>>>>> table1
> > > >>>>>>>>> key
> > > >>>>>>>>>>>> (ie,
> > > >>>>>>>>>>>>>>>>>>>           "A" joins).
> > > >>>>>>>>>>>>>>>>>>>           > If
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key as
> > key
> > > of
> > > >>>>>>> table1,
> > > >>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>           implies that the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> > > join(X,1)>
> > > >> or
> > > >>>>> <A,
> > > >>>>>>>>>>>> join(X,2)>
> > > >>>>>>>>>>>>>>>>>>>           but not
> > > >>>>>>>>>>>>>>>>>>>           > both.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same
> key,
> > > >>>>> whatever
> > > >>>>>>>>> result
> > > >>>>>>>>>>>> record
> > > >>>>>>>>>>>>>>>>>>>           we emit
> > > >>>>>>>>>>>>>>>>>>>           > later,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous result.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan
> > > originally
> > > >>>>>>> proposed
> > > >>>>>>>>>>>> to use
> > > >>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>           > combination
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the input
> > > tables
> > > >>>>> as
> > > >>>>>>> key
> > > >>>>>>>>> of
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           output table.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the output
> > table
> > > >>>>> unique
> > > >>>>>>> and
> > > >>>>>>>>> we
> > > >>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>           store both in
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> > join(X,1)>,
> > > >> <A-b,
> > > >>>>>>>>>>>> join(X,2)>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan
> Filipiak
> > > >> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could be
> > > >>>>> disregarded.
> > > >>>>>>> The
> > > >>>>>>>>>>>> decision
> > > >>>>>>>>>>>>>>>>>>>           about the
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of the
> > > >> aggregated
> > > >>>>>>> map.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps
> would
> > be
> > > >>>>>>> unpacked
> > > >>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> > > >>>>>>>>>>>>>>>>>>>           > element
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as delete.
> > Any
> > > >>>>> other
> > > >>>>>>> count
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in "waiting
> > for
> > > >>>>> correct
> > > >>>>>>>>>>>> deletes to
> > > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam
> > > Bellemare
> > > >>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could
> > replace
> > > >> the
> > > >>>>>>> second
> > > >>>>>>>>>>>>>>>>>>>           repartition store
> > > >>>>>>>>>>>>>>>>>>>           > and
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a
> groupBy
> > > and
> > > >>>>>>> reduce.
> > > >>>>>>>>>>>> However,
> > > >>>>>>>>>>>>>>>>>>>           it looks
> > > >>>>>>>>>>>>>>>>>>>           > like
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the
> > > highwater
> > > >>>>> value
> > > >>>>>>>>> within
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           materialized
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> > > out-of-order
> > > >>>>>>> records
> > > >>>>>>>>>>>> (assuming
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> my
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This in
> > > effect
> > > >> is
> > > >>>>>>> the
> > > >>>>>>>>> same
> > > >>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>           design I
> > > >>>>>>>>>>>>>>>>>>>           > have
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables
> merged
> > > >>>>> together.
> > > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>           >
> > > >>>>>>>>>>>>>>>>>>>           >
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>> -- Guozhang
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>> -- Guozhang
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > > >
> > > >
> > >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
1) I believe that the resolution mechanism John has proposed is sufficient
- it is clean and easy and doesn't require additional RocksDB stores, which
reduces the footprint greatly. I don't think we need to resolve based on
timestamp or offset anymore, but if we decide to do to that would be within
the bounds of the existing API.

2) Is the current API sufficient, or does it need to be altered to go back
to vote?

3) KScatteredTable implementation can always be added in a future revision.
This API does not rule it out. This implementation of this function would
simply be replaced with `KScatteredTable.resolve()` while still maintaining
the existing API, thereby giving both features as Jan outlined earlier.
Would this work?


Thanks Guozhang, John and Jan




On Mon, Dec 10, 2018 at 10:39 AM John Roesler <jo...@confluent.io> wrote:

> Hi, all,
>
> >> In fact, we
> >> can just keep a single final-result store with timestamps and reject
> values
> >> that have a smaller timestamp, is that right?
>
> > Which is the correct output should at least be decided on the offset of
> > the original message.
>
> Thanks for this point, Jan.
>
> KIP-258 is merely to allow embedding the record timestamp  in the k/v
> store,
> as well as providing a storage-format upgrade path.
>
> I might have missed it, but I think we have yet to discuss whether it's
> safe
> or desirable just to swap topic-ordering our for timestamp-ordering. This
> is
> a very deep topic, and I think it would only pollute the current
> discussion.
>
> What Adam has proposed is safe, given the *current* ordering semantics
> of the system. If we can agree on his proposal, I think we can merge the
> feature well before the conversation about timestamp ordering even takes
> place, much less reaches a conclusion. In the mean time, it would seem to
> be unfortunate to have one join operator with different ordering semantics
> from every other KTable operator.
>
> If and when that timestamp discussion takes place, many (all?) KTable
> operations
> will need to be updated, rendering the many:one join a small marginal cost.
>
> And, just to plug it again, I proposed an algorithm above that I believe
> provides
> correct ordering without any additional metadata, and regardless of the
> ordering semantics. I didn't bring it up further, because I felt the KIP
> only needs
> to agree on the public API, and we can discuss the implementation at
> leisure in
> a PR...
>
> Thanks,
> -John
>
>
> On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> >
> >
> > On 10.12.2018 07:42, Guozhang Wang wrote:
> > > Hello Adam / Jan / John,
> > >
> > > Sorry for being late on this thread! I've finally got some time this
> > > weekend to cleanup a load of tasks on my queue (actually I've also
> > realized
> > > there are a bunch of other things I need to enqueue while cleaning them
> > up
> > > --- sth I need to improve on my side). So here are my thoughts:
> > >
> > > Regarding the APIs: I like the current written API in the KIP. More
> > > generally I'd prefer to keep the 1) one-to-many join functionalities as
> > > well as 2) other join types than inner as separate KIPs since 1) may
> > worth
> > > a general API refactoring that can benefit not only foreignkey joins
> but
> > > collocate joins as well (e.g. an extended proposal of
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > ),
> > > and I'm not sure if other join types would actually be needed (maybe
> left
> > > join still makes sense), so it's better to
> wait-for-people-to-ask-and-add
> > > than add-sth-that-no-one-uses.
> > >
> > > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > > KScatteredTable for users to inject their own optimization: I'd prefer
> to
> > > do the current option as-is, and my main rationale is for optimization
> > > rooms inside the Streams internals and the API succinctness. For
> advanced
> > > users who may indeed prefer KScatteredTable and do their own
> > optimization,
> > > while it is too much of the work to use Processor API directly, I think
> > we
> > > can still extend the current API to support it in the future if it
> > becomes
> > > necessary.
> >
> > no internal optimization potential. it's a myth
> >
> > ¯\_(ツ)_/¯
> >
> > :-)
> >
> > >
> > > Another note about step 4) resolving out-of-ordering data, as I
> mentioned
> > > before I think with KIP-258 (embedded timestamp with key-value store)
> we
> > > can actually make this step simpler than the current proposal. In fact,
> > we
> > > can just keep a single final-result store with timestamps and reject
> > values
> > > that have a smaller timestamp, is that right?
> >
> > Which is the correct output should at least be decided on the offset of
> > the original message.
> >
> > >
> > >
> > > That's all I have in mind now. Again, great appreciation to Adam to
> make
> > > such HUGE progress on this KIP!
> > >
> > >
> > > Guozhang
> > >
> > > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <Ja...@trivago.com>
> > > wrote:
> > >
> > >> If they don't find the time:
> > >> They usually take the opposite path from me :D
> > >> so the answer would be clear.
> > >>
> > >> hence my suggestion to vote.
> > >>
> > >>
> > >> On 04.12.2018 21:06, Adam Bellemare wrote:
> > >>> Hi Guozhang and Matthias
> > >>>
> > >>> I know both of you are quite busy, but we've gotten this KIP to a
> point
> > >>> where we need more guidance on the API (perhaps a bit of a
> tie-breaker,
> > >> if
> > >>> you will). If you have anyone else you may think should look at this,
> > >>> please tag them accordingly.
> > >>>
> > >>> The scenario is as such:
> > >>>
> > >>> Current Option:
> > >>> API:
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > >>> 1) Rekey the data to CombinedKey, and shuffles it to the partition
> with
> > >> the
> > >>> foreignKey (repartition 1)
> > >>> 2) Join the data
> > >>> 3) Shuffle the data back to the original node (repartition 2)
> > >>> 4) Resolve out-of-order arrival / race condition due to foreign-key
> > >> changes.
> > >>>
> > >>> Alternate Option:
> > >>> Perform #1 and #2 above, and return a KScatteredTable.
> > >>> - It would be keyed on a wrapped key function: <CombinedKey<KO, K>,
> VR>
> > >> (KO
> > >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> > >>> - KScatteredTable.resolve() would perform #3 and #4 but otherwise a
> > user
> > >>> would be able to perform additional functions directly from the
> > >>> KScatteredTable (TBD - currently out of scope).
> > >>> - John's analysis 2-emails up is accurate as to the tradeoffs.
> > >>>
> > >>> Current Option is coded as-is. Alternate option is possible, but will
> > >>> require for implementation details to be made in the API and some
> > >> exposure
> > >>> of new data structures into the API (ie: CombinedKey).
> > >>>
> > >>> I appreciate any insight into this.
> > >>>
> > >>> Thanks.
> > >>>
> > >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> > adam.bellemare@gmail.com>
> > >>> wrote:
> > >>>
> > >>>> Hi John
> > >>>>
> > >>>> Thanks for your feedback and assistance. I think your summary is
> > >> accurate
> > >>>> from my perspective. Additionally, I would like to add that there
> is a
> > >> risk
> > >>>> of inconsistent final states without performing the resolution. This
> > is
> > >> a
> > >>>> major concern for me as most of the data I have dealt with is
> produced
> > >> by
> > >>>> relational databases. We have seen a number of cases where a user in
> > the
> > >>>> Rails UI has modified the field (foreign key), realized they made a
> > >>>> mistake, and then updated the field again with a new key. The events
> > are
> > >>>> propagated out as they are produced, and as such we have had
> > real-world
> > >>>> cases where these inconsistencies were propagated downstream as the
> > >> final
> > >>>> values due to the race conditions in the fanout of the data.
> > >>>>
> > >>>> This solution that I propose values correctness of the final result
> > over
> > >>>> other factors.
> > >>>>
> > >>>> We could always move this function over to using a KScatteredTable
> > >>>> implementation in the future, and simply deprecate it this join API
> in
> > >>>> time. I think I would like to hear more from some of the other major
> > >>>> committers on which course of action they would think is best before
> > any
> > >>>> more coding is done.
> > >>>>
> > >>>> Thanks again
> > >>>>
> > >>>> Adam
> > >>>>
> > >>>>
> > >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io>
> > wrote:
> > >>>>
> > >>>>> Hi Jan and Adam,
> > >>>>>
> > >>>>> Wow, thanks for doing that test, Adam. Those results are
> encouraging.
> > >>>>>
> > >>>>> Thanks for your performance experience as well, Jan. I agree that
> > >> avoiding
> > >>>>> unnecessary join outputs is especially important when the fan-out
> is
> > so
> > >>>>> high. I suppose this could also be built into the implementation
> > we're
> > >>>>> discussing, but it wouldn't have to be specified in the KIP (since
> > >> it's an
> > >>>>> API-transparent optimization).
> > >>>>>
> > >>>>> As far as whether or not to re-repartition the data, I didn't bring
> > it
> > >> up
> > >>>>> because it sounded like the two of you agreed to leave the KIP
> as-is,
> > >>>>> despite the disagreement.
> > >>>>>
> > >>>>> If you want my opinion, I feel like both approaches are reasonable.
> > >>>>> It sounds like Jan values more the potential for developers to
> > optimize
> > >>>>> their topologies to re-use the intermediate nodes, whereas Adam
> > places
> > >>>>> more
> > >>>>> value on having a single operator that people can use without extra
> > >> steps
> > >>>>> at the end.
> > >>>>>
> > >>>>> Personally, although I do find it exceptionally annoying when a
> > >> framework
> > >>>>> gets in my way when I'm trying to optimize something, it seems
> better
> > >> to
> > >>>>> go
> > >>>>> for a single operation.
> > >>>>> * Encapsulating the internal transitions gives us significant
> > latitude
> > >> in
> > >>>>> the implementation (for example, joining only at the end, not in
> the
> > >>>>> middle
> > >>>>> to avoid extra data copying and out-of-order resolution; how we
> > >> represent
> > >>>>> the first repartition keys (combined keys vs. value vectors),
> etc.).
> > >> If we
> > >>>>> publish something like a KScatteredTable with the right-partitioned
> > >> joined
> > >>>>> data, then the API pretty much locks in the implementation as well.
> > >>>>> * The API seems simpler to understand and use. I do mean "seems";
> if
> > >>>>> anyone
> > >>>>> wants to make the case that KScatteredTable is actually simpler, I
> > >> think
> > >>>>> hypothetical usage code would help. From a relational algebra
> > >> perspective,
> > >>>>> it seems like KTable.join(KTable) should produce a new KTable in
> all
> > >>>>> cases.
> > >>>>> * That said, there might still be room in the API for a different
> > >>>>> operation
> > >>>>> like what Jan has proposed to scatter a KTable, and then do things
> > like
> > >>>>> join, re-group, etc from there... I'm not sure; I haven't thought
> > >> through
> > >>>>> all the consequences yet.
> > >>>>>
> > >>>>> This is all just my opinion after thinking over the discussion so
> > >> far...
> > >>>>> -John
> > >>>>>
> > >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> > >> adam.bellemare@gmail.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Updated the PR to take into account John's feedback.
> > >>>>>>
> > >>>>>> I did some preliminary testing for the performance of the
> > prefixScan.
> > >> I
> > >>>>>> have attached the file, but I will also include the text in the
> body
> > >>>>> here
> > >>>>>> for archival purposes (I am not sure what happens to attached
> > files).
> > >> I
> > >>>>>> also updated the PR and the KIP accordingly.
> > >>>>>>
> > >>>>>> Summary: It scales exceptionally well for scanning large values of
> > >>>>>> records. As Jan mentioned previously, the real issue would be more
> > >>>>> around
> > >>>>>> processing the resulting records after obtaining them. For
> instance,
> > >> it
> > >>>>>> takes approximately ~80-120 mS to flush the buffer and a further
> > >>>>> ~35-85mS
> > >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them.
> Iterating
> > >>>>>> through the records just to generate a simple count takes ~ 40
> times
> > >>>>> longer
> > >>>>>> than the flush + scan combined.
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>> Setup:
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m,
> > >> Xms512m)
> > >>>>>> CPU: i7 2.2 Ghz.
> > >>>>>>
> > >>>>>> Note: I am using a slightly-modified, directly-accessible Kafka
> > >> Streams
> > >>>>>> RocksDB
> > >>>>>> implementation (RocksDB.java, basically just avoiding the
> > >>>>>> ProcessorContext).
> > >>>>>> There are no modifications to the default RocksDB values provided
> in
> > >> the
> > >>>>>> 2.1/trunk release.
> > >>>>>>
> > >>>>>>
> > >>>>>> keysize = 128 bytes
> > >>>>>> valsize = 512 bytes
> > >>>>>>
> > >>>>>> Step 1:
> > >>>>>> Write X positive matching events: (key = prefix + left-padded
> > >>>>>> auto-incrementing integer)
> > >>>>>> Step 2:
> > >>>>>> Write 10X negative matching events (key = left-padded
> > >> auto-incrementing
> > >>>>>> integer)
> > >>>>>> Step 3:
> > >>>>>> Perform flush
> > >>>>>> Step 4:
> > >>>>>> Perform prefixScan
> > >>>>>> Step 5:
> > >>>>>> Iterate through return Iterator and validate the count of expected
> > >>>>> events.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>> Results:
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> ============================================================================================
> > >>>>>> X = 1k (11k events total)
> > >>>>>> Flush Time = 39 mS
> > >>>>>> Scan Time = 7 mS
> > >>>>>> 6.9 MB disk
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>> X = 10k (110k events total)
> > >>>>>> Flush Time = 45 mS
> > >>>>>> Scan Time = 8 mS
> > >>>>>> 127 MB
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>> X = 100k (1.1M events total)
> > >>>>>> Test1:
> > >>>>>> Flush Time = 60 mS
> > >>>>>> Scan Time = 12 mS
> > >>>>>> 678 MB
> > >>>>>>
> > >>>>>> Test2:
> > >>>>>> Flush Time = 45 mS
> > >>>>>> Scan Time = 7 mS
> > >>>>>> 576 MB
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>> X = 1MB (11M events total)
> > >>>>>> Test1:
> > >>>>>> Flush Time = 52 mS
> > >>>>>> Scan Time = 19 mS
> > >>>>>> 7.2 GB
> > >>>>>>
> > >>>>>> Test2:
> > >>>>>> Flush Time = 84 mS
> > >>>>>> Scan Time = 34 mS
> > >>>>>> 9.1 GB
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>> X = 2.5M (27.5M events total)
> > >>>>>> Test1:
> > >>>>>> Flush Time = 82 mS
> > >>>>>> Scan Time = 63 mS
> > >>>>>> 17GB - 276 sst files
> > >>>>>>
> > >>>>>> Test2:
> > >>>>>> Flush Time = 116 mS
> > >>>>>> Scan Time = 35 mS
> > >>>>>> 23GB - 361 sst files
> > >>>>>>
> > >>>>>> Test3:
> > >>>>>> Flush Time = 103 mS
> > >>>>>> Scan Time = 82 mS
> > >>>>>> 19 GB - 300 sst files
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>
> >
> --------------------------------------------------------------------------------------------
> > >>>>>>
> > >>>>>> I had to limit my testing on my laptop to X = 2.5M events. I tried
> > to
> > >> go
> > >>>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+
> range
> > >>>>> and my
> > >>>>>> laptop ran out of disk. More extensive testing could be done but I
> > >>>>> suspect
> > >>>>>> that it would be in line with what we're seeing in the results
> > above.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> At this point in time, I think the only major discussion point is
> > >> really
> > >>>>>> around what Jan and I have disagreed on: repartitioning back +
> > >> resolving
> > >>>>>> potential out of order issues or leaving that up to the client to
> > >>>>> handle.
> > >>>>>>
> > >>>>>>
> > >>>>>> Thanks folks,
> > >>>>>>
> > >>>>>> Adam
> > >>>>>>
> > >>>>>>
> > >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> > Jan.Filipiak@trivago.com
> > >>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> > >>>>>>>> Hi all,
> > >>>>>>>>
> > >>>>>>>> Sorry that this discussion petered out... I think the 2.1
> release
> > >>>>>>> caused an
> > >>>>>>>> extended distraction that pushed it off everyone's radar (which
> > was
> > >>>>>>>> precisely Adam's concern). Personally, I've also had some extend
> > >>>>>>>> distractions of my own that kept (and continue to keep) me
> > >>>>> preoccupied.
> > >>>>>>>>
> > >>>>>>>> However, calling for a vote did wake me up, so I guess Jan was
> on
> > >> the
> > >>>>>>> right
> > >>>>>>>> track!
> > >>>>>>>>
> > >>>>>>>> I've gone back and reviewed the whole KIP document and the prior
> > >>>>>>>> discussion, and I'd like to offer a few thoughts:
> > >>>>>>>>
> > >>>>>>>> API Thoughts:
> > >>>>>>>>
> > >>>>>>>> 1. If I read the KIP right, you are proposing a many-to-one
> join.
> > >>>>> Could
> > >>>>>>> we
> > >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the
> > design
> > >>>>>>> around
> > >>>>>>>> and make it a oneToManyJoin?
> > >>>>>>>>
> > >>>>>>>> The proposed name "joinOnForeignKey" disguises the join type,
> and
> > it
> > >>>>>>> seems
> > >>>>>>>> like it might trick some people into using it for a one-to-one
> > join.
> > >>>>>>> This
> > >>>>>>>> would work, of course, but it would be super inefficient
> compared
> > to
> > >>>>> a
> > >>>>>>>> simple rekey-and-join.
> > >>>>>>>>
> > >>>>>>>> 2. I might have missed it, but I don't think it's specified
> > whether
> > >>>>>>> it's an
> > >>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
> > >>>>> (neglecting
> > >>>>>>> IQ),
> > >>>>>>>> the rest can be achieved by filtering or by handling it in the
> > >>>>>>> ValueJoiner.
> > >>>>>>>>
> > >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
> > >>>>>>>> 3a. Regarding Serialized: There are a few different paradigms in
> > >>>>> play in
> > >>>>>>>> the Streams API, so it's confusing, but instead of three
> > Serialized
> > >>>>>>> args, I
> > >>>>>>>> think it would be better to have one that allows (optionally)
> > >> setting
> > >>>>>>> the 4
> > >>>>>>>> incoming serdes. The result serde is defined by the
> Materialized.
> > >> The
> > >>>>>>>> incoming serdes can be optional because they might already be
> > >>>>> available
> > >>>>>>> on
> > >>>>>>>> the source KTables, or the default serdes from the config might
> be
> > >>>>>>>> applicable.
> > >>>>>>>>
> > >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't
> > allow
> > >>>>>>> setting
> > >>>>>>>> one, and it seems like it might actually be harmful, since the
> > rekey
> > >>>>>>>> operation needs to produce results that are co-partitioned with
> > the
> > >>>>>>> "other"
> > >>>>>>>> KTable.
> > >>>>>>>>
> > >>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't
> actually
> > >>>>>>> follow
> > >>>>>>>> what Matthias meant about namespacing requiring "deserializing"
> > the
> > >>>>>>> record
> > >>>>>>>> header. The headers are already Strings, so I don't think that
> > >>>>>>>> deserialization is required. If we applied the namespace at
> source
> > >>>>> nodes
> > >>>>>>>> and stripped it at sink nodes, this would be practically no
> > >> overhead.
> > >>>>>>> The
> > >>>>>>>> advantage of the namespace idea is that no public API change wrt
> > >>>>> headers
> > >>>>>>>> needs to happen, and no restrictions need to be placed on users'
> > >>>>>>> headers.
> > >>>>>>>>
> > >>>>>>>> (Although I'm wondering if we can get away without the header at
> > >>>>> all...
> > >>>>>>>> stay tuned)
> > >>>>>>>>
> > >>>>>>>> 5. I also didn't follow the discussion about the HWM table
> growing
> > >>>>>>> without
> > >>>>>>>> bound. As I read it, the HWM table is effectively implementing
> OCC
> > >> to
> > >>>>>>>> resolve the problem you noted with disordering when the rekey is
> > >>>>>>>> reversed... particularly notable when the FK changes. As such,
> it
> > >>>>> only
> > >>>>>>>> needs to track the most recent "version" (the offset in the
> source
> > >>>>>>>> partition) of each key. Therefore, it should have the same
> number
> > of
> > >>>>>>> keys
> > >>>>>>>> as the source table at all times.
> > >>>>>>>>
> > >>>>>>>> I see that you are aware of KIP-258, which I think might be
> > relevant
> > >>>>> in
> > >>>>>>> a
> > >>>>>>>> couple of ways. One: it's just about storing the timestamp in
> the
> > >>>>> state
> > >>>>>>>> store, but the ultimate idea is to effectively use the timestamp
> > as
> > >>>>> an
> > >>>>>>> OCC
> > >>>>>>>> "version" to drop disordered updates. You wouldn't want to use
> the
> > >>>>>>>> timestamp for this operation, but if you were to use a similar
> > >>>>>>> mechanism to
> > >>>>>>>> store the source offset in the store alongside the re-keyed
> > values,
> > >>>>> then
> > >>>>>>>> you could avoid a separate table.
> > >>>>>>>>
> > >>>>>>>> 6. You and Jan have been thinking about this for a long time, so
> > >> I've
> > >>>>>>>> probably missed something here, but I'm wondering if we can
> avoid
> > >> the
> > >>>>>>> HWM
> > >>>>>>>> tracking at all and resolve out-of-order during a final join
> > >>>>> instead...
> > >>>>>>>>
> > >>>>>>>> Let's say we're joining a left table (Integer K: Letter FK,
> (other
> > >>>>>>> data))
> > >>>>>>>> to a right table (Letter K: (some data)).
> > >>>>>>>>
> > >>>>>>>> Left table:
> > >>>>>>>> 1: (A, xyz)
> > >>>>>>>> 2: (B, asd)
> > >>>>>>>>
> > >>>>>>>> Right table:
> > >>>>>>>> A: EntityA
> > >>>>>>>> B: EntityB
> > >>>>>>>>
> > >>>>>>>> We could do a rekey as you proposed with a combined key, but not
> > >>>>>>>> propagating the value at all..
> > >>>>>>>> Rekey table:
> > >>>>>>>> A-1: (dummy value)
> > >>>>>>>> B-2: (dummy value)
> > >>>>>>>>
> > >>>>>>>> Which we then join with the right table to produce:
> > >>>>>>>> A-1: EntityA
> > >>>>>>>> B-2: EntityB
> > >>>>>>>>
> > >>>>>>>> Which gets rekeyed back:
> > >>>>>>>> 1: A, EntityA
> > >>>>>>>> 2: B, EntityB
> > >>>>>>>>
> > >>>>>>>> And finally we do the actual join:
> > >>>>>>>> Result table:
> > >>>>>>>> 1: ((A, xyz), EntityA)
> > >>>>>>>> 2: ((B, asd), EntityB)
> > >>>>>>>>
> > >>>>>>>> The thing is that in that last join, we have the opportunity to
> > >>>>> compare
> > >>>>>>> the
> > >>>>>>>> current FK in the left table with the incoming PK of the right
> > >>>>> table. If
> > >>>>>>>> they don't match, we just drop the event, since it must be
> > outdated.
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets
> updated
> > >> to
> > >>>>>>> (1:
> > >>>>>>>> B, xyz), ultimately yielding a conundrum about whether the final
> > >>>>> state
> > >>>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm
> above,
> > >>>>> you
> > >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz),
> (B,
> > >>>>>>>> EntityB)). It seems like this does give you enough information
> to
> > >>>>> make
> > >>>>>>> the
> > >>>>>>>> right choice, regardless of disordering.
> > >>>>>>>
> > >>>>>>> Will check Adams patch, but this should work. As mentioned often
> I
> > am
> > >>>>>>> not convinced on partitioning back for the user automatically. I
> > >> think
> > >>>>>>> this is the real performance eater ;)
> > >>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> 7. Last thought... I'm a little concerned about the performance
> of
> > >>>>> the
> > >>>>>>>> range scans when records change in the right table. You've said
> > that
> > >>>>>>> you've
> > >>>>>>>> been using the algorithm you presented in production for a
> while.
> > >> Can
> > >>>>>>> you
> > >>>>>>>> give us a sense of the performance characteristics you've
> > observed?
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>> Make it work, make it fast, make it beautiful. The topmost thing
> > here
> > >>>>> is
> > >>>>>>> / was correctness. In practice I do not measure the performance
> of
> > >> the
> > >>>>>>> range scan. Usual cases I run this with is emitting 500k - 1kk
> rows
> > >>>>>>> on a left hand side change. The range scan is just the work you
> > gotta
> > >>>>>>> do, also when you pack your data into different formats, usually
> > the
> > >>>>>>> rocks performance is very tight to the size of the data and we
> > can't
> > >>>>>>> really change that. It is more important for users to prevent
> > useless
> > >>>>>>> updates to begin with. My left hand side is guarded to drop
> changes
> > >>>>> that
> > >>>>>>> are not going to change my join output.
> > >>>>>>>
> > >>>>>>> usually it's:
> > >>>>>>>
> > >>>>>>> drop unused fields and then don't forward if old.equals(new)
> > >>>>>>>
> > >>>>>>> regarding to the performance of creating an iterator for smaller
> > >>>>>>> fanouts, users can still just do a group by first then anyways.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> I could only think of one alternative, but I'm not sure if it's
> > >>>>> better
> > >>>>>>> or
> > >>>>>>>> worse... If the first re-key only needs to preserve the original
> > >> key,
> > >>>>>>> as I
> > >>>>>>>> proposed in #6, then we could store a vector of keys in the
> value:
> > >>>>>>>>
> > >>>>>>>> Left table:
> > >>>>>>>> 1: A,...
> > >>>>>>>> 2: B,...
> > >>>>>>>> 3: A,...
> > >>>>>>>>
> > >>>>>>>> Gets re-keyed:
> > >>>>>>>> A: [1, 3]
> > >>>>>>>> B: [2]
> > >>>>>>>>
> > >>>>>>>> Then, the rhs part of the join would only need a regular
> > single-key
> > >>>>>>> lookup.
> > >>>>>>>> Of course we have to deal with the problem of large values, as
> > >>>>> there's
> > >>>>>>> no
> > >>>>>>>> bound on the number of lhs records that can reference rhs
> records.
> > >>>>>>> Offhand,
> > >>>>>>>> I'd say we could page the values, so when one row is past the
> > >>>>>>> threshold, we
> > >>>>>>>> append the key for the next page. Then in most cases, it would
> be
> > a
> > >>>>>>> single
> > >>>>>>>> key lookup, but for large fan-out updates, it would be one per
> > (max
> > >>>>>>> value
> > >>>>>>>> size)/(avg lhs key size).
> > >>>>>>>>
> > >>>>>>>> This seems more complex, though... Plus, I think there's some
> > extra
> > >>>>>>>> tracking we'd need to do to know when to emit a retraction. For
> > >>>>> example,
> > >>>>>>>> when record 1 is deleted, the re-key table would just have (A:
> > [3]).
> > >>>>>>> Some
> > >>>>>>>> kind of tombstone is needed so that the join result for 1 can
> also
> > >> be
> > >>>>>>>> retracted.
> > >>>>>>>>
> > >>>>>>>> That's all!
> > >>>>>>>>
> > >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP.
> Sorry
> > >> the
> > >>>>>>>> discussion has been slow.
> > >>>>>>>> -John
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> > >>>>> Jan.Filipiak@trivago.com>
> > >>>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Id say you can just call the vote.
> > >>>>>>>>>
> > >>>>>>>>> that happens all the time, and if something comes up, it just
> > goes
> > >>>>> back
> > >>>>>>>>> to discuss.
> > >>>>>>>>>
> > >>>>>>>>> would not expect to much attention with another another email
> in
> > >>>>> this
> > >>>>>>>>> thread.
> > >>>>>>>>>
> > >>>>>>>>> best Jan
> > >>>>>>>>>
> > >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> > >>>>>>>>>> Hello Contributors
> > >>>>>>>>>>
> > >>>>>>>>>> I know that 2.1 is about to be released, but I do need to bump
> > >>>>> this to
> > >>>>>>>>> keep
> > >>>>>>>>>> visibility up. I am still intending to push this through once
> > >>>>>>> contributor
> > >>>>>>>>>> feedback is given.
> > >>>>>>>>>>
> > >>>>>>>>>> Main points that need addressing:
> > >>>>>>>>>> 1) Any way (or benefit) in structuring the current singular
> > graph
> > >>>>> node
> > >>>>>>>>> into
> > >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I
> am
> > a
> > >>>>> bit
> > >>>>>>>>> fuzzy
> > >>>>>>>>>> on how the optimizations are supposed to work, so I would
> > >>>>> appreciate
> > >>>>>>> any
> > >>>>>>>>>> help on this aspect.
> > >>>>>>>>>>
> > >>>>>>>>>> 2) Overall strategy for joining + resolving. This thread has
> > much
> > >>>>>>>>> discourse
> > >>>>>>>>>> between Jan and I between the current highwater mark proposal
> > and
> > >> a
> > >>>>>>>>> groupBy
> > >>>>>>>>>> + reduce proposal. I am of the opinion that we need to
> strictly
> > >>>>> handle
> > >>>>>>>>> any
> > >>>>>>>>>> chance of out-of-order data and leave none of it up to the
> > >>>>> consumer.
> > >>>>>>> Any
> > >>>>>>>>>> comments or suggestions here would also help.
> > >>>>>>>>>>
> > >>>>>>>>>> 3) Anything else that you see that would prevent this from
> > moving
> > >>>>> to a
> > >>>>>>>>> vote?
> > >>>>>>>>>>
> > >>>>>>>>>> Thanks
> > >>>>>>>>>>
> > >>>>>>>>>> Adam
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> > >>>>>>>>> adam.bellemare@gmail.com>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Hi Jan
> > >>>>>>>>>>>
> > >>>>>>>>>>> With the Stores.windowStoreBuilder and
> > >>>>> Stores.persistentWindowStore,
> > >>>>>>> you
> > >>>>>>>>>>> actually only need to specify the amount of segments you want
> > and
> > >>>>> how
> > >>>>>>>>> large
> > >>>>>>>>>>> they are. To the best of my understanding, what happens is
> that
> > >>>>> the
> > >>>>>>>>>>> segments are automatically rolled over as new data with new
> > >>>>>>> timestamps
> > >>>>>>>>> are
> > >>>>>>>>>>> created. We use this exact functionality in some of the work
> > done
> > >>>>>>>>>>> internally at my company. For reference, this is the hopping
> > >>>>> windowed
> > >>>>>>>>> store.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>
> >
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> > >>>>>>>>>>>
> > >>>>>>>>>>> In the code that I have provided, there are going to be two
> 24h
> > >>>>>>>>> segments.
> > >>>>>>>>>>> When a record is put into the windowStore, it will be
> inserted
> > at
> > >>>>>>> time
> > >>>>>>>>> T in
> > >>>>>>>>>>> both segments. The two segments will always overlap by 12h.
> As
> > >>>>> time
> > >>>>>>>>> goes on
> > >>>>>>>>>>> and new records are added (say at time T+12h+), the oldest
> > >> segment
> > >>>>>>> will
> > >>>>>>>>> be
> > >>>>>>>>>>> automatically deleted and a new segment created. The records
> > are
> > >>>>> by
> > >>>>>>>>> default
> > >>>>>>>>>>> inserted with the context.timestamp(), such that it is the
> > record
> > >>>>>>> time,
> > >>>>>>>>> not
> > >>>>>>>>>>> the clock time, which is used.
> > >>>>>>>>>>>
> > >>>>>>>>>>> To the best of my understanding, the timestamps are retained
> > when
> > >>>>>>>>>>> restoring from the changelog.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
> > >>>>>>> segment-level,
> > >>>>>>>>>>> instead of at an individual record level.
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> > >>>>>>> Jan.Filipiak@trivago.com>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Will that work? I expected it to blow up with
> > ClassCastException
> > >>>>> or
> > >>>>>>>>>>>> similar.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> You either would have to specify the window you fetch/put or
> > >>>>> iterate
> > >>>>>>>>>>>> across all windows the key was found in right?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I just hope the window-store doesn't check stream-time under
> > the
> > >>>>>>> hoods
> > >>>>>>>>>>>> that would be a questionable interface.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> If it does: did you see my comment on checking all the
> windows
> > >>>>>>> earlier?
> > >>>>>>>>>>>> that would be needed to actually give reasonable time
> > gurantees.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Best
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> > >>>>>>>>>>>>> Hi Jan
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the
> > state
> > >>>>>>> store,
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>> the ProcessorSupplier.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> > >>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> @Guozhang
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks for the information. This is indeed something that
> > >>>>> will be
> > >>>>>>>>>>>>>>> extremely
> > >>>>>>>>>>>>>>> useful for this KIP.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> @Jan
> > >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I will not
> > be
> > >>>>>>> moving
> > >>>>>>>>>>>> ahead
> > >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution
> as
> > >> you
> > >>>>>>>>>>>> propose.
> > >>>>>>>>>>>>>>> That being said, if you wish to implement it yourself off
> > of
> > >>>>> my
> > >>>>>>>>>>>> current PR
> > >>>>>>>>>>>>>>> and submit it as a competitive alternative, I would be
> more
> > >>>>> than
> > >>>>>>>>>>>> happy to
> > >>>>>>>>>>>>>>> help vet that as an alternate solution. As it stands
> right
> > >>>>> now,
> > >>>>>>> I do
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>> really have more time to invest into alternatives without
> > >>>>> there
> > >>>>>>>>> being
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>> strong indication from the binding voters which they
> would
> > >>>>>>> prefer.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on the
> > >>>>> streams
> > >>>>>>>>> DSL
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>> some time already, otherwise I would have pulled this KIP
> > >>>>> through
> > >>>>>>>>>>>> already.
> > >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed state
> > >>>>> store
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I will come
> > >> back
> > >>>>>>> for
> > >>>>>>>>>>>> final
> > >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the
> > binding
> > >>>>>>> voters
> > >>>>>>>>>>>> could
> > >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
> > >>>>> according
> > >>>>>>> to
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> latest plan:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> > >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store.
> > This
> > >>>>>>> could
> > >>>>>>>>> be
> > >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
> > >>>>> completed.
> > >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already
> > updated
> > >>>>> in
> > >>>>>>> the
> > >>>>>>>>>>>> PR?
> > >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing
> something?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> > >>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong
> > >> link,
> > >>>>>>> as it
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of
> KIP-258
> > >>>>> we do
> > >>>>>>>>>>>> want to
> > >>>>>>>>>>>>>>>> have "handling out-of-order data for source KTable" such
> > >> that
> > >>>>>>>>>>>> instead of
> > >>>>>>>>>>>>>>>> blindly apply the updates to the materialized store,
> i.e.
> > >>>>>>> following
> > >>>>>>>>>>>>>>>> offset
> > >>>>>>>>>>>>>>>> ordering, we will reject updates that are older than the
> > >>>>> current
> > >>>>>>>>>>>> key's
> > >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> > >>>>>>>>> wangguoz@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hello Adam,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step
> > (i.e.
> > >>>>> the
> > >>>>>>>>> high
> > >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a
> window
> > >>>>>>> store),
> > >>>>>>>>> I
> > >>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value store
> > >>>>> (i.e.
> > >>>>>>> only
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
> > >>>>> described
> > >>>>>>> in
> > >>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is
> > that
> > >>>>> we
> > >>>>>>> can
> > >>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its
> timestamp
> > is
> > >>>>>>>>> smaller
> > >>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I think it
> is
> > >>>>> very
> > >>>>>>>>>>>> similar to
> > >>>>>>>>>>>>>>>>> what you have in mind for high watermark based
> filtering,
> > >>>>> while
> > >>>>>>>>> you
> > >>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the joining
> > >> records
> > >>>>>>> are
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> correctly
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> inherited though the whole topology to the final stage.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
> > >>>>>>> non-windowed
> > >>>>>>>>>>>> KTables
> > >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really have a
> > good
> > >>>>>>>>> support
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>> their joins anyways (
> > >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> > >>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
> > >>>>> non-key
> > >>>>>>>>> joins
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> > >>>>>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Hi Guozhang
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
> > >> endlessly
> > >>>>>>> based
> > >>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of (<this
> > >>>>> table
> > >>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> key>,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
> > >>>>>>> differentiate
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> between
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal would
> > be
> > >>>>> to
> > >>>>>>>>>>>> replace
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would
> > allow
> > >>>>> the
> > >>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This
> should
> > >>>>> allow
> > >>>>>>> for
> > >>>>>>>>>>>> all
> > >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
> > >>>>>>> customizable
> > >>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just
> 10
> > >>>>>>> minutes
> > >>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> window,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the trick
> > >> here.
> > >>>>>>> Even
> > >>>>>>>>>>>> if I
> > >>>>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
> > >>>>> optional
> > >>>>>>>>>>>> since I
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a little
> > bit
> > >>>>>>>>>>>> sceptical
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> how to determine the window. So esentially one could
> run
> > >>>>> into
> > >>>>>>>>>>>> problems
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> the rapid change happens near a window border. I will
> > check
> > >>>>> you
> > >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we could
> > >>>>> still
> > >>>>>>>>> check
> > >>>>>>>>>>>>>>>>>> _all_
> > >>>>>>>>>>>>>>>>>> windows on read with not to bad performance impact I
> > >> guess.
> > >>>>>>> Will
> > >>>>>>>>>>>> let
> > >>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
> > >>>>> wouldn't
> > >>>>>>> not
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> > >>>>>>>>> timestamp(B).
> > >>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> we can't expect that.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> @Jan
> > >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks for
> > the
> > >>>>>>>>>>>> diagram, it
> > >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not have
> the
> > >>>>>>> original
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> primary
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> key available, and I can see that if it was available
> > then
> > >>>>> you
> > >>>>>>>>>>>> would be
> > >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That
> being
> > >>>>> said,
> > >>>>>>> I
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> encourage
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity
> for
> > >>>>>>> everyone
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> else.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work.
> > But
> > >>>>> I
> > >>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original
> > >> primary
> > >>>>>>> key,
> > >>>>>>>>> We
> > >>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
> > >> basically
> > >>>>>>> not
> > >>>>>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
> > >>>>> original
> > >>>>>>> DSL
> > >>>>>>>>>>>> its
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my
> end.
> > >>>>> Will
> > >>>>>>>>>>>> finish
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay
> > >> inside
> > >>>>>>> the
> > >>>>>>>>>>>> State
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
> > >>>>> propagated?
> > >>>>>>>>> Isn't
> > >>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state store?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is
> gonna
> > >>>>>>> return
> > >>>>>>>>>>>> `null`
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is
> going
> > to
> > >>>>> be
> > >>>>>>> a
> > >>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store
> > directly
> > >>>>> for
> > >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
> > >>>>> store,
> > >>>>>>>>>>>> satisfying
> > >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
> > >>>>> Windowed
> > >>>>>>>>>>>> store is
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull operation
> > we
> > >>>>>>> would
> > >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the
> > window
> > >>>>>>> store
> > >>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> the values then.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group by
> > >>>>> before
> > >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i think it
> > >> would
> > >>>>>>> help
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> users
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the original
> > >>>>> primary
> > >>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> issue I
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> understand that we do not have a solid foundation to
> > build
> > >>>>> on.
> > >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
> > >>>>>>> unfortunate. I
> > >>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not think
> > its
> > >>>>> a
> > >>>>>>> good
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> decision.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>>>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> > >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> > >>>>>>> dumbreprajakta311@gmail.com
> > >>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           please remove me from this group
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan
> Filipiak
> > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > >>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> > >>>>>>>>>>>>>>>>>>>           >
> > >>>>>>>>>>>>>>>>>>>           > give me some time, will make such a
> chart.
> > >> last
> > >>>>>>> time i
> > >>>>>>>>>>>> didn't
> > >>>>>>>>>>>>>>>>>>>           get along
> > >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your
> charts.
> > >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> > >>>>>>>>>>>>>>>>>>>           >
> > >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare
> wrote:
> > >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of what I
> > >> attempted
> > >>>>> on
> > >>>>>>> the
> > >>>>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           >
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> > >>>>>>>>>>>>>>>>>>>
> > >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> > >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> > >>>>>>>>>>>>>>>>>>>           <
> > >>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> > >>>>>>>>>>>>>>>>>>>
> > >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> > >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the start of
> my
> > own
> > >>>>>>>>>>>> implementation
> > >>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>           this
> > >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not get it
> to
> > >>>>> work I
> > >>>>>>> have
> > >>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>>           discarded the
> > >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if you
> wish
> > to
> > >>>>>>> continue
> > >>>>>>>>>>>> pursuing
> > >>>>>>>>>>>>>>>>>>>           for your
> > >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you please
> > >>>>> create a
> > >>>>>>>>>>>> diagram on
> > >>>>>>>>>>>>>>>>>>>           the KIP
> > >>>>>>>>>>>>>>>>>>>           > > carefully explaining your solution.
> > Please
> > >>>>> feel
> > >>>>>>> free
> > >>>>>>>>> to
> > >>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>           the image I
> > >>>>>>>>>>>>>>>>>>>           > > just posted as a starting point. I am
> > having
> > >>>>>>> trouble
> > >>>>>>>>>>>>>>>>>>>           understanding your
> > >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a
> carefully
> > >>>>>>> constructed
> > >>>>>>>>>>>> diagram
> > >>>>>>>>>>>>>>>>>>>           will clear
> > >>>>>>>>>>>>>>>>>>>           > up
> > >>>>>>>>>>>>>>>>>>>           > > any misunderstandings. Alternately,
> > please
> > >>>>> post a
> > >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> > >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess at what
> > you
> > >>>>>>> mean, and
> > >>>>>>>>>>>> since I
> > >>>>>>>>>>>>>>>>>>>           value my
> > >>>>>>>>>>>>>>>>>>>           > own
> > >>>>>>>>>>>>>>>>>>>           > > time as much as you value yours, I
> > believe
> > >> it
> > >>>>> is
> > >>>>>>> your
> > >>>>>>>>>>>>>>>>>>>           responsibility to
> > >>>>>>>>>>>>>>>>>>>           > > provide an implementation instead of me
> > >>>>> trying to
> > >>>>>>>>> guess.
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > > Adam
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM, Jan
> > Filipiak
> > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > >>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > > wrote:
> > >>>>>>>>>>>>>>>>>>>           > >
> > >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing interested.
> kafka
> > >>>>>>> streams at
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>           point supports
> > >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as both
> > streams
> > >>>>> have
> > >>>>>>> the
> > >>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>> key.
> > >>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a join
> > >> where a
> > >>>>>>> KTable
> > >>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>           KTable can
> > >>>>>>>>>>>>>>>>>>>           > have
> > >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship (1:n). We
> > >> exploit
> > >>>>>>> that
> > >>>>>>>>>>>> rocksdb
> > >>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted (At
> least
> > >>>>>>> exposes an
> > >>>>>>>>>>>> API to
> > >>>>>>>>>>>>>>>>>>>           access the
> > >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted fashion).
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats are well
> > >>>>>>> understood
> > >>>>>>>>> now
> > >>>>>>>>>>>> and we
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > basically
> > >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API Design (
> when
> > >> Adam
> > >>>>>>> sees
> > >>>>>>>>> my
> > >>>>>>>>>>>> newest
> > >>>>>>>>>>>>>>>>>>>           message).
> > >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of
> loosing
> > >>>>> those
> > >>>>>>> kinda
> > >>>>>>>>>>>>>>>>>>>           arguments within
> > >>>>>>>>>>>>>>>>>>>           > the
> > >>>>>>>>>>>>>>>>>>>           > >> streams community and I have no clue
> > why.
> > >> So
> > >>>>> I
> > >>>>>>>>>>>> literally
> > >>>>>>>>>>>>>>>>>>>           can't wait for
> > >>>>>>>>>>>>>>>>>>>           > you
> > >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and give
> > you
> > >>>>>>> opinion on
> > >>>>>>>>>>>> how we
> > >>>>>>>>>>>>>>>>>>>           should
> > >>>>>>>>>>>>>>>>>>>           > design
> > >>>>>>>>>>>>>>>>>>>           > >> the return type of the oneToManyJoin
> and
> > >> how
> > >>>>>>> many
> > >>>>>>>>>>>> power we
> > >>>>>>>>>>>>>>>>>>>           want to give
> > >>>>>>>>>>>>>>>>>>>           > to
> > >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where
> > simplicity
> > >>>>> isn't
> > >>>>>>>>>>>> really that
> > >>>>>>>>>>>>>>>>>>>           as users
> > >>>>>>>>>>>>>>>>>>>           > still
> > >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on the
> > >> discussion
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>
> > >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I found
> this
> > >>>>> subject
> > >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> > >>>>>>>>>>>>>>>>>>>           > like
> > >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a join
> > table of
> > >>>>> two
> > >>>>>>>>>>>> streams? Is
> > >>>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>           > somewhere
> > >>>>>>>>>>>>>>>>>>>           > >>> I can see the original requirement or
> > >>>>> proposal?
> > >>>>>>>>>>>>>>>>>>>           > >>>
> > >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan
> > Filipiak
> > >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> > >>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam Bellemare
> > >> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a
> > Windowed
> > >>>>> Store
> > >>>>>>> to
> > >>>>>>>>>>>> store the
> > >>>>>>>>>>>>>>>>>>>           highwater
> > >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> > >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this should work
> > >> fine,
> > >>>>>>> with
> > >>>>>>>>> the
> > >>>>>>>>>>>> caveat
> > >>>>>>>>>>>>>>>>>>>           being that
> > >>>>>>>>>>>>>>>>>>>           > it
> > >>>>>>>>>>>>>>>>>>>           > >>>>> can
> > >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order arrival
> > for up
> > >>>>> to
> > >>>>>>> the
> > >>>>>>>>>>>> size of
> > >>>>>>>>>>>>>>>>>>>           the window
> > >>>>>>>>>>>>>>>>>>>           > (ie:
> > >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would remove
> the
> > >>>>>>> possibility
> > >>>>>>>>>>>> of it
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > unbounded
> > >>>>>>>>>>>>>>>>>>>           > >>>>> in
> > >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's suggestion, I
> > >>>>> believe
> > >>>>>>> this
> > >>>>>>>>> is
> > >>>>>>>>>>>> where
> > >>>>>>>>>>>>>>>>>>>           we will
> > >>>>>>>>>>>>>>>>>>>           > have
> > >>>>>>>>>>>>>>>>>>>           > >>>>> to
> > >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement. While I do
> > not
> > >>>>>>> disagree
> > >>>>>>>>>>>> with your
> > >>>>>>>>>>>>>>>>>>>           statement
> > >>>>>>>>>>>>>>>>>>>           > >>>>> about
> > >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional joins
> > done
> > >>>>> in a
> > >>>>>>>>>>>> real-world
> > >>>>>>>>>>>>>>>>>>>           workflow, I
> > >>>>>>>>>>>>>>>>>>>           > do
> > >>>>>>>>>>>>>>>>>>>           > >>>>> not
> > >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively deal
> > with
> > >>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>> arrival
> > >>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> > >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent joins. I
> have
> > >>>>>>> attempted
> > >>>>>>>>> what
> > >>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>           think you have
> > >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a high-water,
> using
> > >>>>>>> groupBy and
> > >>>>>>>>>>>> reduce)
> > >>>>>>>>>>>>>>>>>>>           and found
> > >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> > >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too
> quickly,
> > or
> > >>>>> the
> > >>>>>>> load
> > >>>>>>>>> on
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>           stream thread
> > >>>>>>>>>>>>>>>>>>>           > is
> > >>>>>>>>>>>>>>>>>>>           > >>>>> too
> > >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages will
> arrive
> > >>>>>>>>> out-of-order
> > >>>>>>>>>>>> and be
> > >>>>>>>>>>>>>>>>>>>           incorrectly
> > >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an
> intermediate
> > >>>>> event
> > >>>>>>> is
> > >>>>>>>>>>>>>>>>>>> represented
> > >>>>>>>>>>>>>>>>>>>           as the
> > >>>>>>>>>>>>>>>>>>>           > final
> > >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on your
> > groupBy
> > >>>>>>>>>>>> implementation.
> > >>>>>>>>>>>>>>>>>>>           There must be
> > >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> > >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it is, I
> > would
> > >>>>> just
> > >>>>>>> like
> > >>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>           confirm. The idea
> > >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
> > >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I would
> > like
> > >> to
> > >>>>>>> clarify
> > >>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>           we draw a
> > >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the scattered
> events
> > >>>>> back to
> > >>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I know how
> > to
> > >>>>>>>>> conclusively
> > >>>>>>>>>>>> deal
> > >>>>>>>>>>>>>>>>>>>           with
> > >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a given time
> > >> frame,
> > >>>>>>> and to
> > >>>>>>>>>>>> ensure
> > >>>>>>>>>>>>>>>>>>>           that the
> > >>>>>>>>>>>>>>>>>>>           > data
> > >>>>>>>>>>>>>>>>>>>           > >>>>> is
> > >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with the
> input
> > >>>>> events.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to share that
> > >>>>>>> illustrates
> > >>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>           approach, I
> > >>>>>>>>>>>>>>>>>>>           > would
> > >>>>>>>>>>>>>>>>>>>           > >>>>> be
> > >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would remove
> any
> > >>>>>>>>>>>> misunderstandings
> > >>>>>>>>>>>>>>>>>>>           that I may
> > >>>>>>>>>>>>>>>>>>>           > have.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for my
> code.
> > I
> > >>>>> don't
> > >>>>>>> have
> > >>>>>>>>>>>>>>>>>>>           something easily
> > >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated with
> > >>>>> OO-patterns.
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> > >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey, V
> > value, T
> > >>>>>>>>> aggregate)
> > >>>>>>>>>>>>>>>>>>>           > >>>>      {
> > >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V>
> currentStateAsMap =
> > >>>>>>>>>>>> asMap(aggregate);
> > >>>>>>>>>>>>>>>>>>> <<
> > >>>>>>>>>>>>>>>>>>>           imaginary
> > >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> > >>>>> mapper.apply(value);
> > >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the place
> > where
> > >>>>> people
> > >>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>           gonna have
> > >>>>>>>>>>>>>>>>>>>           > issues
> > >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't do it.
> > we
> > >>>>> would
> > >>>>>>> need
> > >>>>>>>>>>>> to find
> > >>>>>>>>>>>>>>>>>>>           a solution
> > >>>>>>>>>>>>>>>>>>>           > here.
> > >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> > >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate the
> > field in
> > >>>>> the
> > >>>>>>>>>>>> joiner, so
> > >>>>>>>>>>>>>>>>>>>           that we can
> > >>>>>>>>>>>>>>>>>>>           > pick
> > >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate. Probably you
> > have
> > >>>>> not
> > >>>>>>>>> thought
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>           this in your
> > >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> > >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open to
> > find a
> > >>>>>>> generic
> > >>>>>>>>>>>> solution
> > >>>>>>>>>>>>>>>>>>>           here. In my
> > >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken in
> > >>>>>>>>> KTableImpl.GroupBy
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>           looses
> > >>>>>>>>>>>>>>>>>>>           > the keys
> > >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the aggregate
> key.
> > >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted it away
> > back
> > >>>>>>> then way
> > >>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>> i
> > >>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>           > thinking
> > >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is why I
> > didn't
> > >>>>>>> realize
> > >>>>>>>>> its
> > >>>>>>>>>>>>>>>>>>>           significance here.
> > >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> currentStateAsMap.put(mapper.apply(m),
> > >> m);
> > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > >>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(toModifyKey,
> > >> value);
> > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > >>>>>>>>>>>>>>>>>>>           > >>>>          else
> > >>>>>>>>>>>>>>>>>>>           > >>>>          {
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> currentStateAsMap.remove(toModifyKey);
> > >>>>>>>>>>>>>>>>>>>           > >>>> if(currentStateAsMap.isEmpty()){
> > >>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
> > >>>>>>>>>>>>>>>>>>>           > >>>>              }
> > >>>>>>>>>>>>>>>>>>>           > >>>>          }
> > >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> > >>>>>>> asAggregateType(currentStateAsMap)
> > >>>>>>>>>>>>>>>>>>>           > >>>>      }
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
> > >>>>> Filipiak
> > >>>>>>> <
> > >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
> > >>>>>>>>> Jan.Filipiak@trivago.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing Matthias
> to
> > >>>>> speed!
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I think
> > >> re-keying
> > >>>>>>> back
> > >>>>>>>>>>>> should be
> > >>>>>>>>>>>>>>>>>>>           optional at
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> > KScatteredTable
> > >>>>> with
> > >>>>>>>>>>>> reshuffle()
> > >>>>>>>>>>>>>>>>>>>           returning
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined> to make
> > the
> > >>>>>>> backwards
> > >>>>>>>>>>>>>>>>>>>           repartitioning
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of doing
> > the
> > >>>>> out
> > >>>>>>> of
> > >>>>>>>>> order
> > >>>>>>>>>>>>>>>>>>>           processing using
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark
> tracking.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded growth is
> > just
> > >>>>> scary
> > >>>>>>> + It
> > >>>>>>>>>>>> saves
> > >>>>>>>>>>>>>>>>>>> us
> > >>>>>>>>>>>>>>>>>>>           the header
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of always
> > >>>>>>> repartitioning
> > >>>>>>>>>>>> back is
> > >>>>>>>>>>>>>>>>>>>           just not so
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has been
> done
> > >>>>> before
> > >>>>>>> we
> > >>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>           back and
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards is
> > really
> > >>>>>>> common.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam
> Bellemare
> > >>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your feedback, I do
> > >>>>>>> appreciate
> > >>>>>>>>> it!
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be
> > possible,
> > >> it
> > >>>>>>> would
> > >>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>           > deserialize
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > runtime
> > >>>>>>> overhead.
> > >>>>>>>>> I
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>           suggest to
> > >>>>>>>>>>>>>>>>>>>           > no
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> > >>>>> overhead.
> > >>>>>>> If
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> becomes a
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > problem in
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> name
> > >>>>> spacing
> > >>>>>>>>> later
> > >>>>>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with using a
> > >> reserved
> > >>>>>>> string
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>           document it.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the design
> it
> > >> the
> > >>>>>>> type of
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           result KTable:
> > >>>>>>>>>>>>>>>>>>>           > If
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal
> correctly,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have table1
> > and
> > >>>>> table2
> > >>>>>>>>>>>> swapped.
> > >>>>>>>>>>>>>>>>>>>           Here is how it
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records that
> > contain
> > >>>>> the
> > >>>>>>>>>>>> foreign key
> > >>>>>>>>>>>>>>>>>>>           within their
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> > <a,(fk=A,bar=1)>,
> > >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>, <B,Y>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is required to
> > >> extract
> > >>>>>>> the
> > >>>>>>>>>>>> foreign
> > >>>>>>>>>>>>>>>>>>> key.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper: (
> value
> > =>
> > >>>>>>> value.fk
> > >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to each
> > element
> > >> in
> > >>>>>>>>> table1,
> > >>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>           new combined
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a,
> (fk=A,bar=1)>,
> > >>>>> <A-b,
> > >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> > >>>>>>>>>>>>>>>>>>>           <B-c,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> > >> copartitioned
> > >>>>>>> with
> > >>>>>>>>>>>> table2:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with Partition
> 0:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
> > >>>>> (fk=A,bar=1)>,
> > >>>>>>> <A-b,
> > >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with Partition
> 1:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
> > >> (fk=B,bar=3)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be joined
> > >>>>> together
> > >>>>>>>>> locally
> > >>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>           applying the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design and
> my
> > >>>>> design
> > >>>>>>>>>>>> deviate. My
> > >>>>>>>>>>>>>>>>>>>           design goes
> > >>>>>>>>>>>>>>>>>>>           > on
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data post-join
> and
> > >>>>> resolve
> > >>>>>>>>>>>> out-of-order
> > >>>>>>>>>>>>>>>>>>>           arrival of
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data keyed
> > just
> > >>>>> the
> > >>>>>>>>>>>> original key.
> > >>>>>>>>>>>>>>>>>>>           I do not
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the
> internals
> > >>>>>>> outside of
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make for
> larger
> > >>>>>>> footprint,
> > >>>>>>>>>>>> but it
> > >>>>>>>>>>>>>>>>>>>           removes all
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order
> arrivals
> > >> and
> > >>>>>>>>> handling
> > >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> > >>>>>>>>>>>>>>>>>>>           > the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this makes
> the
> > >>>>>>> function
> > >>>>>>>>> much
> > >>>>>>>>>>>>>>>>>>> easier
> > >>>>>>>>>>>>>>>>>>>           to use.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps resolve
> > your
> > >>>>>>>>> questions,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>           please feel
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your mind.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
> > >>>>> Matthias J.
> > >>>>>>>>> Sax <
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io <mailto:
> > >>>>>>>>>>>> matthias@confluent.io>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on this
> > >> thread. I
> > >>>>>>> did
> > >>>>>>>>> not
> > >>>>>>>>>>>> read
> > >>>>>>>>>>>>>>>>>>>           everything so
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple of
> > initial
> > >>>>>>> thoughts:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is a
> > >> fundamental
> > >>>>>>>>>>>> difference
> > >>>>>>>>>>>>>>>>>>>           between header
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258. For 258,
> > we
> > >> add
> > >>>>>>>>> headers
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>           changelog topic
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams and
> > nobody
> > >>>>>>> else is
> > >>>>>>>>>>>> supposed
> > >>>>>>>>>>>>>>>>>>>           to write
> > >>>>>>>>>>>>>>>>>>>           > into
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user header
> are
> > >>>>> written
> > >>>>>>> into
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           changelog topic
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not conflicts.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see a big
> > issue
> > >>>>> with
> > >>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>>           headers within
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it, we
> can
> > >> have
> > >>>>>>> some
> > >>>>>>>>>>>> "reserved"
> > >>>>>>>>>>>>>>>>>>>           header keys
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to use
> when
> > >>>>>>> processing
> > >>>>>>>>>>>> data with
> > >>>>>>>>>>>>>>>>>>>           Kafka
> > >>>>>>>>>>>>>>>>>>>           > Streams.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe way to
> > avoid
> > >>>>>>>>> conflicts,
> > >>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>>>>>>           > headers
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal topics
> (I
> > >>>>> think):
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and changelog
> > topics,
> > >> we
> > >>>>> can
> > >>>>>>>>>>>> namespace
> > >>>>>>>>>>>>>>>>>>>           all headers:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers are
> > >> namespaced
> > >>>>> as
> > >>>>>>>>>>>> "external."
> > >>>>>>>>>>>>>>>>>>> +
> > >>>>>>>>>>>>>>>>>>>           headerKey
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> > namespaced as
> > >>>>>>>>>>>> "internal." +
> > >>>>>>>>>>>>>>>>>>>           headerKey
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would be
> > >> possible,
> > >>>>> it
> > >>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> > runtime
> > >>>>>>> overhead.
> > >>>>>>>>> I
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>           suggest to
> > >>>>>>>>>>>>>>>>>>>           > no
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> > >>>>> overhead.
> > >>>>>>> If
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> becomes a
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > problem in
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add
> name
> > >>>>> spacing
> > >>>>>>>>> later
> > >>>>>>>>>>>> on.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the design
> > it
> > >> the
> > >>>>>>> type
> > >>>>>>>>> of
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           result KTable:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal
> correctly,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable =
> > >>>>>>>>>>>> table1.join(table2,...);
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the `joinedTable`
> has
> > >> the
> > >>>>>>> same
> > >>>>>>>>> key
> > >>>>>>>>>>>> as the
> > >>>>>>>>>>>>>>>>>>>           left input
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work because
> > if
> > >>>>> table2
> > >>>>>>>>>>>> contains
> > >>>>>>>>>>>>>>>>>>>           multiple rows
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in table1
> > (what is
> > >>>>> the
> > >>>>>>> main
> > >>>>>>>>>>>> purpose
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>           > foreign
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table would
> only
> > >>>>>>> contain a
> > >>>>>>>>>>>> single
> > >>>>>>>>>>>>>>>>>>>           join result,
> > >>>>>>>>>>>>>>>>>>>           > but
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream: <a,(A,1)>,
> > >>>>> <b,(A,2)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a foreign
> key
> > to
> > >>>>>>> table1
> > >>>>>>>>> key
> > >>>>>>>>>>>> (ie,
> > >>>>>>>>>>>>>>>>>>>           "A" joins).
> > >>>>>>>>>>>>>>>>>>>           > If
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key as
> key
> > of
> > >>>>>>> table1,
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>           implies that the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> > join(X,1)>
> > >> or
> > >>>>> <A,
> > >>>>>>>>>>>> join(X,2)>
> > >>>>>>>>>>>>>>>>>>>           but not
> > >>>>>>>>>>>>>>>>>>>           > both.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same key,
> > >>>>> whatever
> > >>>>>>>>> result
> > >>>>>>>>>>>> record
> > >>>>>>>>>>>>>>>>>>>           we emit
> > >>>>>>>>>>>>>>>>>>>           > later,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous result.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan
> > originally
> > >>>>>>> proposed
> > >>>>>>>>>>>> to use
> > >>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>           > combination
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the input
> > tables
> > >>>>> as
> > >>>>>>> key
> > >>>>>>>>> of
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           output table.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the output
> table
> > >>>>> unique
> > >>>>>>> and
> > >>>>>>>>> we
> > >>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>           store both in
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a,
> join(X,1)>,
> > >> <A-b,
> > >>>>>>>>>>>> join(X,2)>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak
> > >> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could be
> > >>>>> disregarded.
> > >>>>>>> The
> > >>>>>>>>>>>> decision
> > >>>>>>>>>>>>>>>>>>>           about the
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of the
> > >> aggregated
> > >>>>>>> map.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps would
> be
> > >>>>>>> unpacked
> > >>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>           forwarded. 0
> > >>>>>>>>>>>>>>>>>>>           > element
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as delete.
> Any
> > >>>>> other
> > >>>>>>> count
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in "waiting
> for
> > >>>>> correct
> > >>>>>>>>>>>> deletes to
> > >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam
> > Bellemare
> > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could
> replace
> > >> the
> > >>>>>>> second
> > >>>>>>>>>>>>>>>>>>>           repartition store
> > >>>>>>>>>>>>>>>>>>>           > and
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a groupBy
> > and
> > >>>>>>> reduce.
> > >>>>>>>>>>>> However,
> > >>>>>>>>>>>>>>>>>>>           it looks
> > >>>>>>>>>>>>>>>>>>>           > like
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the
> > highwater
> > >>>>> value
> > >>>>>>>>> within
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           materialized
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> > out-of-order
> > >>>>>>> records
> > >>>>>>>>>>>> (assuming
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This in
> > effect
> > >> is
> > >>>>>>> the
> > >>>>>>>>> same
> > >>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>           design I
> > >>>>>>>>>>>>>>>>>>>           > have
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables merged
> > >>>>> together.
> > >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>           >
> > >>>>>>>>>>>>>>>>>>>           >
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>> -- Guozhang
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>> -- Guozhang
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> > >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi, all,

>> In fact, we
>> can just keep a single final-result store with timestamps and reject
values
>> that have a smaller timestamp, is that right?

> Which is the correct output should at least be decided on the offset of
> the original message.

Thanks for this point, Jan.

KIP-258 is merely to allow embedding the record timestamp  in the k/v
store,
as well as providing a storage-format upgrade path.

I might have missed it, but I think we have yet to discuss whether it's safe
or desirable just to swap topic-ordering our for timestamp-ordering. This is
a very deep topic, and I think it would only pollute the current discussion.

What Adam has proposed is safe, given the *current* ordering semantics
of the system. If we can agree on his proposal, I think we can merge the
feature well before the conversation about timestamp ordering even takes
place, much less reaches a conclusion. In the mean time, it would seem to
be unfortunate to have one join operator with different ordering semantics
from every other KTable operator.

If and when that timestamp discussion takes place, many (all?) KTable
operations
will need to be updated, rendering the many:one join a small marginal cost.

And, just to plug it again, I proposed an algorithm above that I believe
provides
correct ordering without any additional metadata, and regardless of the
ordering semantics. I didn't bring it up further, because I felt the KIP
only needs
to agree on the public API, and we can discuss the implementation at
leisure in
a PR...

Thanks,
-John


On Mon, Dec 10, 2018 at 2:28 AM Jan Filipiak <Ja...@trivago.com>
wrote:

>
>
> On 10.12.2018 07:42, Guozhang Wang wrote:
> > Hello Adam / Jan / John,
> >
> > Sorry for being late on this thread! I've finally got some time this
> > weekend to cleanup a load of tasks on my queue (actually I've also
> realized
> > there are a bunch of other things I need to enqueue while cleaning them
> up
> > --- sth I need to improve on my side). So here are my thoughts:
> >
> > Regarding the APIs: I like the current written API in the KIP. More
> > generally I'd prefer to keep the 1) one-to-many join functionalities as
> > well as 2) other join types than inner as separate KIPs since 1) may
> worth
> > a general API refactoring that can benefit not only foreignkey joins but
> > collocate joins as well (e.g. an extended proposal of
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> ),
> > and I'm not sure if other join types would actually be needed (maybe left
> > join still makes sense), so it's better to wait-for-people-to-ask-and-add
> > than add-sth-that-no-one-uses.
> >
> > Regarding whether we enforce step 3) / 4) v.s. introducing a
> > KScatteredTable for users to inject their own optimization: I'd prefer to
> > do the current option as-is, and my main rationale is for optimization
> > rooms inside the Streams internals and the API succinctness. For advanced
> > users who may indeed prefer KScatteredTable and do their own
> optimization,
> > while it is too much of the work to use Processor API directly, I think
> we
> > can still extend the current API to support it in the future if it
> becomes
> > necessary.
>
> no internal optimization potential. it's a myth
>
> ¯\_(ツ)_/¯
>
> :-)
>
> >
> > Another note about step 4) resolving out-of-ordering data, as I mentioned
> > before I think with KIP-258 (embedded timestamp with key-value store) we
> > can actually make this step simpler than the current proposal. In fact,
> we
> > can just keep a single final-result store with timestamps and reject
> values
> > that have a smaller timestamp, is that right?
>
> Which is the correct output should at least be decided on the offset of
> the original message.
>
> >
> >
> > That's all I have in mind now. Again, great appreciation to Adam to make
> > such HUGE progress on this KIP!
> >
> >
> > Guozhang
> >
> > On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >> If they don't find the time:
> >> They usually take the opposite path from me :D
> >> so the answer would be clear.
> >>
> >> hence my suggestion to vote.
> >>
> >>
> >> On 04.12.2018 21:06, Adam Bellemare wrote:
> >>> Hi Guozhang and Matthias
> >>>
> >>> I know both of you are quite busy, but we've gotten this KIP to a point
> >>> where we need more guidance on the API (perhaps a bit of a tie-breaker,
> >> if
> >>> you will). If you have anyone else you may think should look at this,
> >>> please tag them accordingly.
> >>>
> >>> The scenario is as such:
> >>>
> >>> Current Option:
> >>> API:
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> >>> 1) Rekey the data to CombinedKey, and shuffles it to the partition with
> >> the
> >>> foreignKey (repartition 1)
> >>> 2) Join the data
> >>> 3) Shuffle the data back to the original node (repartition 2)
> >>> 4) Resolve out-of-order arrival / race condition due to foreign-key
> >> changes.
> >>>
> >>> Alternate Option:
> >>> Perform #1 and #2 above, and return a KScatteredTable.
> >>> - It would be keyed on a wrapped key function: <CombinedKey<KO, K>, VR>
> >> (KO
> >>> = Other Table Key, K = This Table Key, VR = Joined Result)
> >>> - KScatteredTable.resolve() would perform #3 and #4 but otherwise a
> user
> >>> would be able to perform additional functions directly from the
> >>> KScatteredTable (TBD - currently out of scope).
> >>> - John's analysis 2-emails up is accurate as to the tradeoffs.
> >>>
> >>> Current Option is coded as-is. Alternate option is possible, but will
> >>> require for implementation details to be made in the API and some
> >> exposure
> >>> of new data structures into the API (ie: CombinedKey).
> >>>
> >>> I appreciate any insight into this.
> >>>
> >>> Thanks.
> >>>
> >>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <
> adam.bellemare@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi John
> >>>>
> >>>> Thanks for your feedback and assistance. I think your summary is
> >> accurate
> >>>> from my perspective. Additionally, I would like to add that there is a
> >> risk
> >>>> of inconsistent final states without performing the resolution. This
> is
> >> a
> >>>> major concern for me as most of the data I have dealt with is produced
> >> by
> >>>> relational databases. We have seen a number of cases where a user in
> the
> >>>> Rails UI has modified the field (foreign key), realized they made a
> >>>> mistake, and then updated the field again with a new key. The events
> are
> >>>> propagated out as they are produced, and as such we have had
> real-world
> >>>> cases where these inconsistencies were propagated downstream as the
> >> final
> >>>> values due to the race conditions in the fanout of the data.
> >>>>
> >>>> This solution that I propose values correctness of the final result
> over
> >>>> other factors.
> >>>>
> >>>> We could always move this function over to using a KScatteredTable
> >>>> implementation in the future, and simply deprecate it this join API in
> >>>> time. I think I would like to hear more from some of the other major
> >>>> committers on which course of action they would think is best before
> any
> >>>> more coding is done.
> >>>>
> >>>> Thanks again
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io>
> wrote:
> >>>>
> >>>>> Hi Jan and Adam,
> >>>>>
> >>>>> Wow, thanks for doing that test, Adam. Those results are encouraging.
> >>>>>
> >>>>> Thanks for your performance experience as well, Jan. I agree that
> >> avoiding
> >>>>> unnecessary join outputs is especially important when the fan-out is
> so
> >>>>> high. I suppose this could also be built into the implementation
> we're
> >>>>> discussing, but it wouldn't have to be specified in the KIP (since
> >> it's an
> >>>>> API-transparent optimization).
> >>>>>
> >>>>> As far as whether or not to re-repartition the data, I didn't bring
> it
> >> up
> >>>>> because it sounded like the two of you agreed to leave the KIP as-is,
> >>>>> despite the disagreement.
> >>>>>
> >>>>> If you want my opinion, I feel like both approaches are reasonable.
> >>>>> It sounds like Jan values more the potential for developers to
> optimize
> >>>>> their topologies to re-use the intermediate nodes, whereas Adam
> places
> >>>>> more
> >>>>> value on having a single operator that people can use without extra
> >> steps
> >>>>> at the end.
> >>>>>
> >>>>> Personally, although I do find it exceptionally annoying when a
> >> framework
> >>>>> gets in my way when I'm trying to optimize something, it seems better
> >> to
> >>>>> go
> >>>>> for a single operation.
> >>>>> * Encapsulating the internal transitions gives us significant
> latitude
> >> in
> >>>>> the implementation (for example, joining only at the end, not in the
> >>>>> middle
> >>>>> to avoid extra data copying and out-of-order resolution; how we
> >> represent
> >>>>> the first repartition keys (combined keys vs. value vectors), etc.).
> >> If we
> >>>>> publish something like a KScatteredTable with the right-partitioned
> >> joined
> >>>>> data, then the API pretty much locks in the implementation as well.
> >>>>> * The API seems simpler to understand and use. I do mean "seems"; if
> >>>>> anyone
> >>>>> wants to make the case that KScatteredTable is actually simpler, I
> >> think
> >>>>> hypothetical usage code would help. From a relational algebra
> >> perspective,
> >>>>> it seems like KTable.join(KTable) should produce a new KTable in all
> >>>>> cases.
> >>>>> * That said, there might still be room in the API for a different
> >>>>> operation
> >>>>> like what Jan has proposed to scatter a KTable, and then do things
> like
> >>>>> join, re-group, etc from there... I'm not sure; I haven't thought
> >> through
> >>>>> all the consequences yet.
> >>>>>
> >>>>> This is all just my opinion after thinking over the discussion so
> >> far...
> >>>>> -John
> >>>>>
> >>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Updated the PR to take into account John's feedback.
> >>>>>>
> >>>>>> I did some preliminary testing for the performance of the
> prefixScan.
> >> I
> >>>>>> have attached the file, but I will also include the text in the body
> >>>>> here
> >>>>>> for archival purposes (I am not sure what happens to attached
> files).
> >> I
> >>>>>> also updated the PR and the KIP accordingly.
> >>>>>>
> >>>>>> Summary: It scales exceptionally well for scanning large values of
> >>>>>> records. As Jan mentioned previously, the real issue would be more
> >>>>> around
> >>>>>> processing the resulting records after obtaining them. For instance,
> >> it
> >>>>>> takes approximately ~80-120 mS to flush the buffer and a further
> >>>>> ~35-85mS
> >>>>>> to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
> >>>>>> through the records just to generate a simple count takes ~ 40 times
> >>>>> longer
> >>>>>> than the flush + scan combined.
> >>>>>>
> >>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>> Setup:
> >>>>>>
> >>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m,
> >> Xms512m)
> >>>>>> CPU: i7 2.2 Ghz.
> >>>>>>
> >>>>>> Note: I am using a slightly-modified, directly-accessible Kafka
> >> Streams
> >>>>>> RocksDB
> >>>>>> implementation (RocksDB.java, basically just avoiding the
> >>>>>> ProcessorContext).
> >>>>>> There are no modifications to the default RocksDB values provided in
> >> the
> >>>>>> 2.1/trunk release.
> >>>>>>
> >>>>>>
> >>>>>> keysize = 128 bytes
> >>>>>> valsize = 512 bytes
> >>>>>>
> >>>>>> Step 1:
> >>>>>> Write X positive matching events: (key = prefix + left-padded
> >>>>>> auto-incrementing integer)
> >>>>>> Step 2:
> >>>>>> Write 10X negative matching events (key = left-padded
> >> auto-incrementing
> >>>>>> integer)
> >>>>>> Step 3:
> >>>>>> Perform flush
> >>>>>> Step 4:
> >>>>>> Perform prefixScan
> >>>>>> Step 5:
> >>>>>> Iterate through return Iterator and validate the count of expected
> >>>>> events.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>> Results:
> >>>>>>
> >>>>>>
> >>>>>
> >>
> ============================================================================================
> >>>>>> X = 1k (11k events total)
> >>>>>> Flush Time = 39 mS
> >>>>>> Scan Time = 7 mS
> >>>>>> 6.9 MB disk
> >>>>>>
> >>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>> X = 10k (110k events total)
> >>>>>> Flush Time = 45 mS
> >>>>>> Scan Time = 8 mS
> >>>>>> 127 MB
> >>>>>>
> >>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>> X = 100k (1.1M events total)
> >>>>>> Test1:
> >>>>>> Flush Time = 60 mS
> >>>>>> Scan Time = 12 mS
> >>>>>> 678 MB
> >>>>>>
> >>>>>> Test2:
> >>>>>> Flush Time = 45 mS
> >>>>>> Scan Time = 7 mS
> >>>>>> 576 MB
> >>>>>>
> >>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>> X = 1MB (11M events total)
> >>>>>> Test1:
> >>>>>> Flush Time = 52 mS
> >>>>>> Scan Time = 19 mS
> >>>>>> 7.2 GB
> >>>>>>
> >>>>>> Test2:
> >>>>>> Flush Time = 84 mS
> >>>>>> Scan Time = 34 mS
> >>>>>> 9.1 GB
> >>>>>>
> >>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>> X = 2.5M (27.5M events total)
> >>>>>> Test1:
> >>>>>> Flush Time = 82 mS
> >>>>>> Scan Time = 63 mS
> >>>>>> 17GB - 276 sst files
> >>>>>>
> >>>>>> Test2:
> >>>>>> Flush Time = 116 mS
> >>>>>> Scan Time = 35 mS
> >>>>>> 23GB - 361 sst files
> >>>>>>
> >>>>>> Test3:
> >>>>>> Flush Time = 103 mS
> >>>>>> Scan Time = 82 mS
> >>>>>> 19 GB - 300 sst files
> >>>>>>
> >>>>>>
> >>>>>
> >>
> --------------------------------------------------------------------------------------------
> >>>>>>
> >>>>>> I had to limit my testing on my laptop to X = 2.5M events. I tried
> to
> >> go
> >>>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+ range
> >>>>> and my
> >>>>>> laptop ran out of disk. More extensive testing could be done but I
> >>>>> suspect
> >>>>>> that it would be in line with what we're seeing in the results
> above.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> At this point in time, I think the only major discussion point is
> >> really
> >>>>>> around what Jan and I have disagreed on: repartitioning back +
> >> resolving
> >>>>>> potential out of order issues or leaving that up to the client to
> >>>>> handle.
> >>>>>>
> >>>>>>
> >>>>>> Thanks folks,
> >>>>>>
> >>>>>> Adam
> >>>>>>
> >>>>>>
> >>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <
> Jan.Filipiak@trivago.com
> >>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On 29.11.2018 15:14, John Roesler wrote:
> >>>>>>>> Hi all,
> >>>>>>>>
> >>>>>>>> Sorry that this discussion petered out... I think the 2.1 release
> >>>>>>> caused an
> >>>>>>>> extended distraction that pushed it off everyone's radar (which
> was
> >>>>>>>> precisely Adam's concern). Personally, I've also had some extend
> >>>>>>>> distractions of my own that kept (and continue to keep) me
> >>>>> preoccupied.
> >>>>>>>>
> >>>>>>>> However, calling for a vote did wake me up, so I guess Jan was on
> >> the
> >>>>>>> right
> >>>>>>>> track!
> >>>>>>>>
> >>>>>>>> I've gone back and reviewed the whole KIP document and the prior
> >>>>>>>> discussion, and I'd like to offer a few thoughts:
> >>>>>>>>
> >>>>>>>> API Thoughts:
> >>>>>>>>
> >>>>>>>> 1. If I read the KIP right, you are proposing a many-to-one join.
> >>>>> Could
> >>>>>>> we
> >>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the
> design
> >>>>>>> around
> >>>>>>>> and make it a oneToManyJoin?
> >>>>>>>>
> >>>>>>>> The proposed name "joinOnForeignKey" disguises the join type, and
> it
> >>>>>>> seems
> >>>>>>>> like it might trick some people into using it for a one-to-one
> join.
> >>>>>>> This
> >>>>>>>> would work, of course, but it would be super inefficient compared
> to
> >>>>> a
> >>>>>>>> simple rekey-and-join.
> >>>>>>>>
> >>>>>>>> 2. I might have missed it, but I don't think it's specified
> whether
> >>>>>>> it's an
> >>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
> >>>>> (neglecting
> >>>>>>> IQ),
> >>>>>>>> the rest can be achieved by filtering or by handling it in the
> >>>>>>> ValueJoiner.
> >>>>>>>>
> >>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
> >>>>>>>> 3a. Regarding Serialized: There are a few different paradigms in
> >>>>> play in
> >>>>>>>> the Streams API, so it's confusing, but instead of three
> Serialized
> >>>>>>> args, I
> >>>>>>>> think it would be better to have one that allows (optionally)
> >> setting
> >>>>>>> the 4
> >>>>>>>> incoming serdes. The result serde is defined by the Materialized.
> >> The
> >>>>>>>> incoming serdes can be optional because they might already be
> >>>>> available
> >>>>>>> on
> >>>>>>>> the source KTables, or the default serdes from the config might be
> >>>>>>>> applicable.
> >>>>>>>>
> >>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't
> allow
> >>>>>>> setting
> >>>>>>>> one, and it seems like it might actually be harmful, since the
> rekey
> >>>>>>>> operation needs to produce results that are co-partitioned with
> the
> >>>>>>> "other"
> >>>>>>>> KTable.
> >>>>>>>>
> >>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't actually
> >>>>>>> follow
> >>>>>>>> what Matthias meant about namespacing requiring "deserializing"
> the
> >>>>>>> record
> >>>>>>>> header. The headers are already Strings, so I don't think that
> >>>>>>>> deserialization is required. If we applied the namespace at source
> >>>>> nodes
> >>>>>>>> and stripped it at sink nodes, this would be practically no
> >> overhead.
> >>>>>>> The
> >>>>>>>> advantage of the namespace idea is that no public API change wrt
> >>>>> headers
> >>>>>>>> needs to happen, and no restrictions need to be placed on users'
> >>>>>>> headers.
> >>>>>>>>
> >>>>>>>> (Although I'm wondering if we can get away without the header at
> >>>>> all...
> >>>>>>>> stay tuned)
> >>>>>>>>
> >>>>>>>> 5. I also didn't follow the discussion about the HWM table growing
> >>>>>>> without
> >>>>>>>> bound. As I read it, the HWM table is effectively implementing OCC
> >> to
> >>>>>>>> resolve the problem you noted with disordering when the rekey is
> >>>>>>>> reversed... particularly notable when the FK changes. As such, it
> >>>>> only
> >>>>>>>> needs to track the most recent "version" (the offset in the source
> >>>>>>>> partition) of each key. Therefore, it should have the same number
> of
> >>>>>>> keys
> >>>>>>>> as the source table at all times.
> >>>>>>>>
> >>>>>>>> I see that you are aware of KIP-258, which I think might be
> relevant
> >>>>> in
> >>>>>>> a
> >>>>>>>> couple of ways. One: it's just about storing the timestamp in the
> >>>>> state
> >>>>>>>> store, but the ultimate idea is to effectively use the timestamp
> as
> >>>>> an
> >>>>>>> OCC
> >>>>>>>> "version" to drop disordered updates. You wouldn't want to use the
> >>>>>>>> timestamp for this operation, but if you were to use a similar
> >>>>>>> mechanism to
> >>>>>>>> store the source offset in the store alongside the re-keyed
> values,
> >>>>> then
> >>>>>>>> you could avoid a separate table.
> >>>>>>>>
> >>>>>>>> 6. You and Jan have been thinking about this for a long time, so
> >> I've
> >>>>>>>> probably missed something here, but I'm wondering if we can avoid
> >> the
> >>>>>>> HWM
> >>>>>>>> tracking at all and resolve out-of-order during a final join
> >>>>> instead...
> >>>>>>>>
> >>>>>>>> Let's say we're joining a left table (Integer K: Letter FK, (other
> >>>>>>> data))
> >>>>>>>> to a right table (Letter K: (some data)).
> >>>>>>>>
> >>>>>>>> Left table:
> >>>>>>>> 1: (A, xyz)
> >>>>>>>> 2: (B, asd)
> >>>>>>>>
> >>>>>>>> Right table:
> >>>>>>>> A: EntityA
> >>>>>>>> B: EntityB
> >>>>>>>>
> >>>>>>>> We could do a rekey as you proposed with a combined key, but not
> >>>>>>>> propagating the value at all..
> >>>>>>>> Rekey table:
> >>>>>>>> A-1: (dummy value)
> >>>>>>>> B-2: (dummy value)
> >>>>>>>>
> >>>>>>>> Which we then join with the right table to produce:
> >>>>>>>> A-1: EntityA
> >>>>>>>> B-2: EntityB
> >>>>>>>>
> >>>>>>>> Which gets rekeyed back:
> >>>>>>>> 1: A, EntityA
> >>>>>>>> 2: B, EntityB
> >>>>>>>>
> >>>>>>>> And finally we do the actual join:
> >>>>>>>> Result table:
> >>>>>>>> 1: ((A, xyz), EntityA)
> >>>>>>>> 2: ((B, asd), EntityB)
> >>>>>>>>
> >>>>>>>> The thing is that in that last join, we have the opportunity to
> >>>>> compare
> >>>>>>> the
> >>>>>>>> current FK in the left table with the incoming PK of the right
> >>>>> table. If
> >>>>>>>> they don't match, we just drop the event, since it must be
> outdated.
> >>>>>>>>
> >>>>>>>
> >>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets updated
> >> to
> >>>>>>> (1:
> >>>>>>>> B, xyz), ultimately yielding a conundrum about whether the final
> >>>>> state
> >>>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm above,
> >>>>> you
> >>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> >>>>>>>> EntityB)). It seems like this does give you enough information to
> >>>>> make
> >>>>>>> the
> >>>>>>>> right choice, regardless of disordering.
> >>>>>>>
> >>>>>>> Will check Adams patch, but this should work. As mentioned often I
> am
> >>>>>>> not convinced on partitioning back for the user automatically. I
> >> think
> >>>>>>> this is the real performance eater ;)
> >>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 7. Last thought... I'm a little concerned about the performance of
> >>>>> the
> >>>>>>>> range scans when records change in the right table. You've said
> that
> >>>>>>> you've
> >>>>>>>> been using the algorithm you presented in production for a while.
> >> Can
> >>>>>>> you
> >>>>>>>> give us a sense of the performance characteristics you've
> observed?
> >>>>>>>>
> >>>>>>>
> >>>>>>> Make it work, make it fast, make it beautiful. The topmost thing
> here
> >>>>> is
> >>>>>>> / was correctness. In practice I do not measure the performance of
> >> the
> >>>>>>> range scan. Usual cases I run this with is emitting 500k - 1kk rows
> >>>>>>> on a left hand side change. The range scan is just the work you
> gotta
> >>>>>>> do, also when you pack your data into different formats, usually
> the
> >>>>>>> rocks performance is very tight to the size of the data and we
> can't
> >>>>>>> really change that. It is more important for users to prevent
> useless
> >>>>>>> updates to begin with. My left hand side is guarded to drop changes
> >>>>> that
> >>>>>>> are not going to change my join output.
> >>>>>>>
> >>>>>>> usually it's:
> >>>>>>>
> >>>>>>> drop unused fields and then don't forward if old.equals(new)
> >>>>>>>
> >>>>>>> regarding to the performance of creating an iterator for smaller
> >>>>>>> fanouts, users can still just do a group by first then anyways.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>> I could only think of one alternative, but I'm not sure if it's
> >>>>> better
> >>>>>>> or
> >>>>>>>> worse... If the first re-key only needs to preserve the original
> >> key,
> >>>>>>> as I
> >>>>>>>> proposed in #6, then we could store a vector of keys in the value:
> >>>>>>>>
> >>>>>>>> Left table:
> >>>>>>>> 1: A,...
> >>>>>>>> 2: B,...
> >>>>>>>> 3: A,...
> >>>>>>>>
> >>>>>>>> Gets re-keyed:
> >>>>>>>> A: [1, 3]
> >>>>>>>> B: [2]
> >>>>>>>>
> >>>>>>>> Then, the rhs part of the join would only need a regular
> single-key
> >>>>>>> lookup.
> >>>>>>>> Of course we have to deal with the problem of large values, as
> >>>>> there's
> >>>>>>> no
> >>>>>>>> bound on the number of lhs records that can reference rhs records.
> >>>>>>> Offhand,
> >>>>>>>> I'd say we could page the values, so when one row is past the
> >>>>>>> threshold, we
> >>>>>>>> append the key for the next page. Then in most cases, it would be
> a
> >>>>>>> single
> >>>>>>>> key lookup, but for large fan-out updates, it would be one per
> (max
> >>>>>>> value
> >>>>>>>> size)/(avg lhs key size).
> >>>>>>>>
> >>>>>>>> This seems more complex, though... Plus, I think there's some
> extra
> >>>>>>>> tracking we'd need to do to know when to emit a retraction. For
> >>>>> example,
> >>>>>>>> when record 1 is deleted, the re-key table would just have (A:
> [3]).
> >>>>>>> Some
> >>>>>>>> kind of tombstone is needed so that the join result for 1 can also
> >> be
> >>>>>>>> retracted.
> >>>>>>>>
> >>>>>>>> That's all!
> >>>>>>>>
> >>>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry
> >> the
> >>>>>>>> discussion has been slow.
> >>>>>>>> -John
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> >>>>> Jan.Filipiak@trivago.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Id say you can just call the vote.
> >>>>>>>>>
> >>>>>>>>> that happens all the time, and if something comes up, it just
> goes
> >>>>> back
> >>>>>>>>> to discuss.
> >>>>>>>>>
> >>>>>>>>> would not expect to much attention with another another email in
> >>>>> this
> >>>>>>>>> thread.
> >>>>>>>>>
> >>>>>>>>> best Jan
> >>>>>>>>>
> >>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> >>>>>>>>>> Hello Contributors
> >>>>>>>>>>
> >>>>>>>>>> I know that 2.1 is about to be released, but I do need to bump
> >>>>> this to
> >>>>>>>>> keep
> >>>>>>>>>> visibility up. I am still intending to push this through once
> >>>>>>> contributor
> >>>>>>>>>> feedback is given.
> >>>>>>>>>>
> >>>>>>>>>> Main points that need addressing:
> >>>>>>>>>> 1) Any way (or benefit) in structuring the current singular
> graph
> >>>>> node
> >>>>>>>>> into
> >>>>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I am
> a
> >>>>> bit
> >>>>>>>>> fuzzy
> >>>>>>>>>> on how the optimizations are supposed to work, so I would
> >>>>> appreciate
> >>>>>>> any
> >>>>>>>>>> help on this aspect.
> >>>>>>>>>>
> >>>>>>>>>> 2) Overall strategy for joining + resolving. This thread has
> much
> >>>>>>>>> discourse
> >>>>>>>>>> between Jan and I between the current highwater mark proposal
> and
> >> a
> >>>>>>>>> groupBy
> >>>>>>>>>> + reduce proposal. I am of the opinion that we need to strictly
> >>>>> handle
> >>>>>>>>> any
> >>>>>>>>>> chance of out-of-order data and leave none of it up to the
> >>>>> consumer.
> >>>>>>> Any
> >>>>>>>>>> comments or suggestions here would also help.
> >>>>>>>>>>
> >>>>>>>>>> 3) Anything else that you see that would prevent this from
> moving
> >>>>> to a
> >>>>>>>>> vote?
> >>>>>>>>>>
> >>>>>>>>>> Thanks
> >>>>>>>>>>
> >>>>>>>>>> Adam
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >>>>>>>>> adam.bellemare@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Jan
> >>>>>>>>>>>
> >>>>>>>>>>> With the Stores.windowStoreBuilder and
> >>>>> Stores.persistentWindowStore,
> >>>>>>> you
> >>>>>>>>>>> actually only need to specify the amount of segments you want
> and
> >>>>> how
> >>>>>>>>> large
> >>>>>>>>>>> they are. To the best of my understanding, what happens is that
> >>>>> the
> >>>>>>>>>>> segments are automatically rolled over as new data with new
> >>>>>>> timestamps
> >>>>>>>>> are
> >>>>>>>>>>> created. We use this exact functionality in some of the work
> done
> >>>>>>>>>>> internally at my company. For reference, this is the hopping
> >>>>> windowed
> >>>>>>>>> store.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>>>>>>>>>>
> >>>>>>>>>>> In the code that I have provided, there are going to be two 24h
> >>>>>>>>> segments.
> >>>>>>>>>>> When a record is put into the windowStore, it will be inserted
> at
> >>>>>>> time
> >>>>>>>>> T in
> >>>>>>>>>>> both segments. The two segments will always overlap by 12h. As
> >>>>> time
> >>>>>>>>> goes on
> >>>>>>>>>>> and new records are added (say at time T+12h+), the oldest
> >> segment
> >>>>>>> will
> >>>>>>>>> be
> >>>>>>>>>>> automatically deleted and a new segment created. The records
> are
> >>>>> by
> >>>>>>>>> default
> >>>>>>>>>>> inserted with the context.timestamp(), such that it is the
> record
> >>>>>>> time,
> >>>>>>>>> not
> >>>>>>>>>>> the clock time, which is used.
> >>>>>>>>>>>
> >>>>>>>>>>> To the best of my understanding, the timestamps are retained
> when
> >>>>>>>>>>> restoring from the changelog.
> >>>>>>>>>>>
> >>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
> >>>>>>> segment-level,
> >>>>>>>>>>> instead of at an individual record level.
> >>>>>>>>>>>
> >>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >>>>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Will that work? I expected it to blow up with
> ClassCastException
> >>>>> or
> >>>>>>>>>>>> similar.
> >>>>>>>>>>>>
> >>>>>>>>>>>> You either would have to specify the window you fetch/put or
> >>>>> iterate
> >>>>>>>>>>>> across all windows the key was found in right?
> >>>>>>>>>>>>
> >>>>>>>>>>>> I just hope the window-store doesn't check stream-time under
> the
> >>>>>>> hoods
> >>>>>>>>>>>> that would be a questionable interface.
> >>>>>>>>>>>>
> >>>>>>>>>>>> If it does: did you see my comment on checking all the windows
> >>>>>>> earlier?
> >>>>>>>>>>>> that would be needed to actually give reasonable time
> gurantees.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>>>>>>>>>>> Hi Jan
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the
> state
> >>>>>>> store,
> >>>>>>>>>>>> not
> >>>>>>>>>>>>> the ProcessorSupplier.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> @Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for the information. This is indeed something that
> >>>>> will be
> >>>>>>>>>>>>>>> extremely
> >>>>>>>>>>>>>>> useful for this KIP.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>> Thanks for your explanations. That being said, I will not
> be
> >>>>>>> moving
> >>>>>>>>>>>> ahead
> >>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution as
> >> you
> >>>>>>>>>>>> propose.
> >>>>>>>>>>>>>>> That being said, if you wish to implement it yourself off
> of
> >>>>> my
> >>>>>>>>>>>> current PR
> >>>>>>>>>>>>>>> and submit it as a competitive alternative, I would be more
> >>>>> than
> >>>>>>>>>>>> happy to
> >>>>>>>>>>>>>>> help vet that as an alternate solution. As it stands right
> >>>>> now,
> >>>>>>> I do
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>> really have more time to invest into alternatives without
> >>>>> there
> >>>>>>>>> being
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>> strong indication from the binding voters which they would
> >>>>>>> prefer.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on the
> >>>>> streams
> >>>>>>>>> DSL
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>> some time already, otherwise I would have pulled this KIP
> >>>>> through
> >>>>>>>>>>>> already.
> >>>>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed state
> >>>>> store
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>> week or so, exercising it via tests, and then I will come
> >> back
> >>>>>>> for
> >>>>>>>>>>>> final
> >>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the
> binding
> >>>>>>> voters
> >>>>>>>>>>>> could
> >>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
> >>>>> according
> >>>>>>> to
> >>>>>>>>> the
> >>>>>>>>>>>>>>> latest plan:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store.
> This
> >>>>>>> could
> >>>>>>>>> be
> >>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
> >>>>> completed.
> >>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already
> updated
> >>>>> in
> >>>>>>> the
> >>>>>>>>>>>> PR?
> >>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing something?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> >>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong
> >> link,
> >>>>>>> as it
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258
> >>>>> we do
> >>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>> have "handling out-of-order data for source KTable" such
> >> that
> >>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>> blindly apply the updates to the materialized store, i.e.
> >>>>>>> following
> >>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>> ordering, we will reject updates that are older than the
> >>>>> current
> >>>>>>>>>>>> key's
> >>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> >>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hello Adam,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step
> (i.e.
> >>>>> the
> >>>>>>>>> high
> >>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a window
> >>>>>>> store),
> >>>>>>>>> I
> >>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value store
> >>>>> (i.e.
> >>>>>>> only
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
> >>>>> described
> >>>>>>> in
> >>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is
> that
> >>>>> we
> >>>>>>> can
> >>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>> "reject" updates from the source topics if its timestamp
> is
> >>>>>>>>> smaller
> >>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>> the current key's latest update timestamp. I think it is
> >>>>> very
> >>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>> what you have in mind for high watermark based filtering,
> >>>>> while
> >>>>>>>>> you
> >>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>> need to make sure that the timestamps of the joining
> >> records
> >>>>>>> are
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> correctly
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> inherited though the whole topology to the final stage.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
> >>>>>>> non-windowed
> >>>>>>>>>>>> KTables
> >>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really have a
> good
> >>>>>>>>> support
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> their joins anyways (
> >>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
> >>>>> non-key
> >>>>>>>>> joins
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >>>>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Guozhang
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
> >> endlessly
> >>>>>>> based
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of (<this
> >>>>> table
> >>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> key>,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
> >>>>>>> differentiate
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal would
> be
> >>>>> to
> >>>>>>>>>>>> replace
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would
> allow
> >>>>> the
> >>>>>>>>> same
> >>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This should
> >>>>> allow
> >>>>>>> for
> >>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
> >>>>>>> customizable
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
> >>>>>>> minutes
> >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> window,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the trick
> >> here.
> >>>>>>> Even
> >>>>>>>>>>>> if I
> >>>>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
> >>>>> optional
> >>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a little
> bit
> >>>>>>>>>>>> sceptical
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> how to determine the window. So esentially one could run
> >>>>> into
> >>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> the rapid change happens near a window border. I will
> check
> >>>>> you
> >>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we could
> >>>>> still
> >>>>>>>>> check
> >>>>>>>>>>>>>>>>>> _all_
> >>>>>>>>>>>>>>>>>> windows on read with not to bad performance impact I
> >> guess.
> >>>>>>> Will
> >>>>>>>>>>>> let
> >>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
> >>>>> wouldn't
> >>>>>>> not
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> >>>>>>>>> timestamp(B).
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> we can't expect that.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks for
> the
> >>>>>>>>>>>> diagram, it
> >>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not have the
> >>>>>>> original
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> key available, and I can see that if it was available
> then
> >>>>> you
> >>>>>>>>>>>> would be
> >>>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That being
> >>>>> said,
> >>>>>>> I
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> encourage
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity for
> >>>>>>> everyone
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> else.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work.
> But
> >>>>> I
> >>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original
> >> primary
> >>>>>>> key,
> >>>>>>>>> We
> >>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
> >> basically
> >>>>>>> not
> >>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
> >>>>> original
> >>>>>>> DSL
> >>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my end.
> >>>>> Will
> >>>>>>>>>>>> finish
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay
> >> inside
> >>>>>>> the
> >>>>>>>>>>>> State
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
> >>>>> propagated?
> >>>>>>>>> Isn't
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state store?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
> >>>>>>> return
> >>>>>>>>>>>> `null`
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is going
> to
> >>>>> be
> >>>>>>> a
> >>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store
> directly
> >>>>> for
> >>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
> >>>>> store,
> >>>>>>>>>>>> satisfying
> >>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
> >>>>> Windowed
> >>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> keeping the values, so for the next statefull operation
> we
> >>>>>>> would
> >>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the
> window
> >>>>>>> store
> >>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> the values then.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group by
> >>>>> before
> >>>>>>>>>>>>>>>>>> repartitioning to the original primary key i think it
> >> would
> >>>>>>> help
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> big time in building efficient apps. Given the original
> >>>>> primary
> >>>>>>>>> key
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> issue I
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> understand that we do not have a solid foundation to
> build
> >>>>> on.
> >>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
> >>>>>>> unfortunate. I
> >>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not think
> its
> >>>>> a
> >>>>>>> good
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> decision.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >>>>>>> dumbreprajakta311@gmail.com
> >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           please remove me from this group
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> >>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > Hi Adam,
> >>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>           > give me some time, will make such a chart.
> >> last
> >>>>>>> time i
> >>>>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>>>>           get along
> >>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your charts.
> >>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
> >>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>>>           > > Hi Jan
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > > I have included a diagram of what I
> >> attempted
> >>>>> on
> >>>>>>> the
> >>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>>>>>>>>>>>
> >>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>>>>>>>>>>           <
> >>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>>>>>>>>>>>
> >>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > > I attempted this back at the start of my
> own
> >>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>           this
> >>>>>>>>>>>>>>>>>>>           > > solution, and since I could not get it to
> >>>>> work I
> >>>>>>> have
> >>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>           discarded the
> >>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if you wish
> to
> >>>>>>> continue
> >>>>>>>>>>>> pursuing
> >>>>>>>>>>>>>>>>>>>           for your
> >>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you please
> >>>>> create a
> >>>>>>>>>>>> diagram on
> >>>>>>>>>>>>>>>>>>>           the KIP
> >>>>>>>>>>>>>>>>>>>           > > carefully explaining your solution.
> Please
> >>>>> feel
> >>>>>>> free
> >>>>>>>>> to
> >>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>           the image I
> >>>>>>>>>>>>>>>>>>>           > > just posted as a starting point. I am
> having
> >>>>>>> trouble
> >>>>>>>>>>>>>>>>>>>           understanding your
> >>>>>>>>>>>>>>>>>>>           > > explanations but I think that a carefully
> >>>>>>> constructed
> >>>>>>>>>>>> diagram
> >>>>>>>>>>>>>>>>>>>           will clear
> >>>>>>>>>>>>>>>>>>>           > up
> >>>>>>>>>>>>>>>>>>>           > > any misunderstandings. Alternately,
> please
> >>>>> post a
> >>>>>>>>>>>>>>>>>>>           comprehensive PR with
> >>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess at what
> you
> >>>>>>> mean, and
> >>>>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>>>           value my
> >>>>>>>>>>>>>>>>>>>           > own
> >>>>>>>>>>>>>>>>>>>           > > time as much as you value yours, I
> believe
> >> it
> >>>>> is
> >>>>>>> your
> >>>>>>>>>>>>>>>>>>>           responsibility to
> >>>>>>>>>>>>>>>>>>>           > > provide an implementation instead of me
> >>>>> trying to
> >>>>>>>>> guess.
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > > Adam
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM, Jan
> Filipiak
> >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> >>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > > wrote:
> >>>>>>>>>>>>>>>>>>>           > >
> >>>>>>>>>>>>>>>>>>>           > >> Hi James,
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing interested. kafka
> >>>>>>> streams at
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>           point supports
> >>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as both
> streams
> >>>>> have
> >>>>>>> the
> >>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a join
> >> where a
> >>>>>>> KTable
> >>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>           KTable can
> >>>>>>>>>>>>>>>>>>>           > have
> >>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship (1:n). We
> >> exploit
> >>>>>>> that
> >>>>>>>>>>>> rocksdb
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted (At least
> >>>>>>> exposes an
> >>>>>>>>>>>> API to
> >>>>>>>>>>>>>>>>>>>           access the
> >>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted fashion).
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats are well
> >>>>>>> understood
> >>>>>>>>> now
> >>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > basically
> >>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API Design ( when
> >> Adam
> >>>>>>> sees
> >>>>>>>>> my
> >>>>>>>>>>>> newest
> >>>>>>>>>>>>>>>>>>>           message).
> >>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of loosing
> >>>>> those
> >>>>>>> kinda
> >>>>>>>>>>>>>>>>>>>           arguments within
> >>>>>>>>>>>>>>>>>>>           > the
> >>>>>>>>>>>>>>>>>>>           > >> streams community and I have no clue
> why.
> >> So
> >>>>> I
> >>>>>>>>>>>> literally
> >>>>>>>>>>>>>>>>>>>           can't wait for
> >>>>>>>>>>>>>>>>>>>           > you
> >>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and give
> you
> >>>>>>> opinion on
> >>>>>>>>>>>> how we
> >>>>>>>>>>>>>>>>>>>           should
> >>>>>>>>>>>>>>>>>>>           > design
> >>>>>>>>>>>>>>>>>>>           > >> the return type of the oneToManyJoin and
> >> how
> >>>>>>> many
> >>>>>>>>>>>> power we
> >>>>>>>>>>>>>>>>>>>           want to give
> >>>>>>>>>>>>>>>>>>>           > to
> >>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where
> simplicity
> >>>>> isn't
> >>>>>>>>>>>> really that
> >>>>>>>>>>>>>>>>>>>           as users
> >>>>>>>>>>>>>>>>>>>           > still
> >>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on the
> >> discussion
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >> Best Jan
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>>>>>>>>>>>>>>>>>           > >>
> >>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I found this
> >>>>> subject
> >>>>>>>>>>>>>>>>>>>           interesting.  Sounds
> >>>>>>>>>>>>>>>>>>>           > like
> >>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a join
> table of
> >>>>> two
> >>>>>>>>>>>> streams? Is
> >>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>           > somewhere
> >>>>>>>>>>>>>>>>>>>           > >>> I can see the original requirement or
> >>>>> proposal?
> >>>>>>>>>>>>>>>>>>>           > >>>
> >>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan
> Filipiak
> >>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
> >>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam Bellemare
> >> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a
> Windowed
> >>>>> Store
> >>>>>>> to
> >>>>>>>>>>>> store the
> >>>>>>>>>>>>>>>>>>>           highwater
> >>>>>>>>>>>>>>>>>>>           > >>>>> mark.
> >>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this should work
> >> fine,
> >>>>>>> with
> >>>>>>>>> the
> >>>>>>>>>>>> caveat
> >>>>>>>>>>>>>>>>>>>           being that
> >>>>>>>>>>>>>>>>>>>           > it
> >>>>>>>>>>>>>>>>>>>           > >>>>> can
> >>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order arrival
> for up
> >>>>> to
> >>>>>>> the
> >>>>>>>>>>>> size of
> >>>>>>>>>>>>>>>>>>>           the window
> >>>>>>>>>>>>>>>>>>>           > (ie:
> >>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would remove the
> >>>>>>> possibility
> >>>>>>>>>>>> of it
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > unbounded
> >>>>>>>>>>>>>>>>>>>           > >>>>> in
> >>>>>>>>>>>>>>>>>>>           > >>>>> size.
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's suggestion, I
> >>>>> believe
> >>>>>>> this
> >>>>>>>>> is
> >>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>           we will
> >>>>>>>>>>>>>>>>>>>           > have
> >>>>>>>>>>>>>>>>>>>           > >>>>> to
> >>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement. While I do
> not
> >>>>>>> disagree
> >>>>>>>>>>>> with your
> >>>>>>>>>>>>>>>>>>>           statement
> >>>>>>>>>>>>>>>>>>>           > >>>>> about
> >>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional joins
> done
> >>>>> in a
> >>>>>>>>>>>> real-world
> >>>>>>>>>>>>>>>>>>>           workflow, I
> >>>>>>>>>>>>>>>>>>>           > do
> >>>>>>>>>>>>>>>>>>>           > >>>>> not
> >>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively deal
> with
> >>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
> >>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent joins. I have
> >>>>>>> attempted
> >>>>>>>>> what
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>           think you have
> >>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a high-water, using
> >>>>>>> groupBy and
> >>>>>>>>>>>> reduce)
> >>>>>>>>>>>>>>>>>>>           and found
> >>>>>>>>>>>>>>>>>>>           > >>>>> that if
> >>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too quickly,
> or
> >>>>> the
> >>>>>>> load
> >>>>>>>>> on
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>           stream thread
> >>>>>>>>>>>>>>>>>>>           > is
> >>>>>>>>>>>>>>>>>>>           > >>>>> too
> >>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages will arrive
> >>>>>>>>> out-of-order
> >>>>>>>>>>>> and be
> >>>>>>>>>>>>>>>>>>>           incorrectly
> >>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an intermediate
> >>>>> event
> >>>>>>> is
> >>>>>>>>>>>>>>>>>>> represented
> >>>>>>>>>>>>>>>>>>>           as the
> >>>>>>>>>>>>>>>>>>>           > final
> >>>>>>>>>>>>>>>>>>>           > >>>>> event.
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on your
> groupBy
> >>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>>>>           There must be
> >>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
> >>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it is, I
> would
> >>>>> just
> >>>>>>> like
> >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>           confirm. The idea
> >>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
> >>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I would
> like
> >> to
> >>>>>>> clarify
> >>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>           we draw a
> >>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the scattered events
> >>>>> back to
> >>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I know how
> to
> >>>>>>>>> conclusively
> >>>>>>>>>>>> deal
> >>>>>>>>>>>>>>>>>>>           with
> >>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a given time
> >> frame,
> >>>>>>> and to
> >>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>           that the
> >>>>>>>>>>>>>>>>>>>           > data
> >>>>>>>>>>>>>>>>>>>           > >>>>> is
> >>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with the input
> >>>>> events.
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to share that
> >>>>>>> illustrates
> >>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>           approach, I
> >>>>>>>>>>>>>>>>>>>           > would
> >>>>>>>>>>>>>>>>>>>           > >>>>> be
> >>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would remove any
> >>>>>>>>>>>> misunderstandings
> >>>>>>>>>>>>>>>>>>>           that I may
> >>>>>>>>>>>>>>>>>>>           > have.
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for my code.
> I
> >>>>> don't
> >>>>>>> have
> >>>>>>>>>>>>>>>>>>>           something easily
> >>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated with
> >>>>> OO-patterns.
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> @Override
> >>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey, V
> value, T
> >>>>>>>>> aggregate)
> >>>>>>>>>>>>>>>>>>>           > >>>>      {
> >>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V> currentStateAsMap =
> >>>>>>>>>>>> asMap(aggregate);
> >>>>>>>>>>>>>>>>>>> <<
> >>>>>>>>>>>>>>>>>>>           imaginary
> >>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
> >>>>> mapper.apply(value);
> >>>>>>>>>>>>>>>>>>>           > >>>>              << this is the place
> where
> >>>>> people
> >>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>           gonna have
> >>>>>>>>>>>>>>>>>>>           > issues
> >>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't do it.
> we
> >>>>> would
> >>>>>>> need
> >>>>>>>>>>>> to find
> >>>>>>>>>>>>>>>>>>>           a solution
> >>>>>>>>>>>>>>>>>>>           > here.
> >>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
> >>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate the
> field in
> >>>>> the
> >>>>>>>>>>>> joiner, so
> >>>>>>>>>>>>>>>>>>>           that we can
> >>>>>>>>>>>>>>>>>>>           > pick
> >>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate. Probably you
> have
> >>>>> not
> >>>>>>>>> thought
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>           this in your
> >>>>>>>>>>>>>>>>>>>           > >>>> approach right?
> >>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open to
> find a
> >>>>>>> generic
> >>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>           here. In my
> >>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken in
> >>>>>>>>> KTableImpl.GroupBy
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>           looses
> >>>>>>>>>>>>>>>>>>>           > the keys
> >>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the aggregate key.
> >>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted it away
> back
> >>>>>>> then way
> >>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>> i
> >>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>           > thinking
> >>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is why I
> didn't
> >>>>>>> realize
> >>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>           significance here.
> >>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
> >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(mapper.apply(m),
> >> m);
> >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
> >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(toModifyKey,
> >> value);
> >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >>>>>>>>>>>>>>>>>>>           > >>>>          else
> >>>>>>>>>>>>>>>>>>>           > >>>>          {
> >>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>>>>>>>>>>           > >>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
> >>>>>>>>>>>>>>>>>>>           > >>>>              }
> >>>>>>>>>>>>>>>>>>>           > >>>>          }
> >>>>>>>>>>>>>>>>>>>           > >>>>          retrun
> >>>>>>> asAggregateType(currentStateAsMap)
> >>>>>>>>>>>>>>>>>>>           > >>>>      }
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>
> >>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
> >>>>>>>>>>>>>>>>>>>           > >>>>> Adam
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
> >>>>> Filipiak
> >>>>>>> <
> >>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
> >>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing Matthias to
> >>>>> speed!
> >>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I think
> >> re-keying
> >>>>>>> back
> >>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>           optional at
> >>>>>>>>>>>>>>>>>>>           > >>>>>> best.
> >>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a
> KScatteredTable
> >>>>> with
> >>>>>>>>>>>> reshuffle()
> >>>>>>>>>>>>>>>>>>>           returning
> >>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined> to make
> the
> >>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>           repartitioning
> >>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
> >>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of doing
> the
> >>>>> out
> >>>>>>> of
> >>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>           processing using
> >>>>>>>>>>>>>>>>>>>           > >>>>>> group
> >>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark tracking.
> >>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded growth is
> just
> >>>>> scary
> >>>>>>> + It
> >>>>>>>>>>>> saves
> >>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>           the header
> >>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of always
> >>>>>>> repartitioning
> >>>>>>>>>>>> back is
> >>>>>>>>>>>>>>>>>>>           just not so
> >>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has been done
> >>>>> before
> >>>>>>> we
> >>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>           back and
> >>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
> >>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards is
> really
> >>>>>>> common.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam Bellemare
> >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your feedback, I do
> >>>>>>> appreciate
> >>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be
> possible,
> >> it
> >>>>>>> would
> >>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>           > deserialize
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> runtime
> >>>>>>> overhead.
> >>>>>>>>> I
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>           suggest to
> >>>>>>>>>>>>>>>>>>>           > no
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> >>>>> overhead.
> >>>>>>> If
> >>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > problem in
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add name
> >>>>> spacing
> >>>>>>>>> later
> >>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with using a
> >> reserved
> >>>>>>> string
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>           document it.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the design it
> >> the
> >>>>>>> type of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           result KTable:
> >>>>>>>>>>>>>>>>>>>           > If
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> I
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have table1
> and
> >>>>> table2
> >>>>>>>>>>>> swapped.
> >>>>>>>>>>>>>>>>>>>           Here is how it
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> works
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records that
> contain
> >>>>> the
> >>>>>>>>>>>> foreign key
> >>>>>>>>>>>>>>>>>>>           within their
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream:
> <a,(fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is required to
> >> extract
> >>>>>>> the
> >>>>>>>>>>>> foreign
> >>>>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper: ( value
> =>
> >>>>>>> value.fk
> >>>>>>>>>>>>>>>>>>>           <http://value.fk> )
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to each
> element
> >> in
> >>>>>>>>> table1,
> >>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>           new combined
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>,
> >>>>> <A-b,
> >>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>>>           <B-c,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
> >> copartitioned
> >>>>>>> with
> >>>>>>>>>>>> table2:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
> >>>>> (fk=A,bar=1)>,
> >>>>>>> <A-b,
> >>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
> >> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be joined
> >>>>> together
> >>>>>>>>> locally
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>           applying the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design and my
> >>>>> design
> >>>>>>>>>>>> deviate. My
> >>>>>>>>>>>>>>>>>>>           design goes
> >>>>>>>>>>>>>>>>>>>           > on
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> to
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data post-join and
> >>>>> resolve
> >>>>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>>>           arrival of
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data keyed
> just
> >>>>> the
> >>>>>>>>>>>> original key.
> >>>>>>>>>>>>>>>>>>>           I do not
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the internals
> >>>>>>> outside of
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           joinOnForeignKey
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make for larger
> >>>>>>> footprint,
> >>>>>>>>>>>> but it
> >>>>>>>>>>>>>>>>>>>           removes all
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order arrivals
> >> and
> >>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>           CombinedKeys from
> >>>>>>>>>>>>>>>>>>>           > the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this makes the
> >>>>>>> function
> >>>>>>>>> much
> >>>>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>>>           to use.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps resolve
> your
> >>>>>>>>> questions,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>           please feel
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your mind.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
> >>>>> Matthias J.
> >>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io <mailto:
> >>>>>>>>>>>> matthias@confluent.io>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on this
> >> thread. I
> >>>>>>> did
> >>>>>>>>> not
> >>>>>>>>>>>> read
> >>>>>>>>>>>>>>>>>>>           everything so
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple of
> initial
> >>>>>>> thoughts:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is a
> >> fundamental
> >>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>           between header
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258. For 258,
> we
> >> add
> >>>>>>>>> headers
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>           changelog topic
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams and
> nobody
> >>>>>>> else is
> >>>>>>>>>>>> supposed
> >>>>>>>>>>>>>>>>>>>           to write
> >>>>>>>>>>>>>>>>>>>           > into
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user header are
> >>>>> written
> >>>>>>> into
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           changelog topic
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not conflicts.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see a big
> issue
> >>>>> with
> >>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>           headers within
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it, we can
> >> have
> >>>>>>> some
> >>>>>>>>>>>> "reserved"
> >>>>>>>>>>>>>>>>>>>           header keys
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to use when
> >>>>>>> processing
> >>>>>>>>>>>> data with
> >>>>>>>>>>>>>>>>>>>           Kafka
> >>>>>>>>>>>>>>>>>>>           > Streams.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe way to
> avoid
> >>>>>>>>> conflicts,
> >>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>           > headers
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal topics (I
> >>>>> think):
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and changelog
> topics,
> >> we
> >>>>> can
> >>>>>>>>>>>> namespace
> >>>>>>>>>>>>>>>>>>>           all headers:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers are
> >> namespaced
> >>>>> as
> >>>>>>>>>>>> "external."
> >>>>>>>>>>>>>>>>>>> +
> >>>>>>>>>>>>>>>>>>>           headerKey
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are
> namespaced as
> >>>>>>>>>>>> "internal." +
> >>>>>>>>>>>>>>>>>>>           headerKey
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would be
> >> possible,
> >>>>> it
> >>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a
> runtime
> >>>>>>> overhead.
> >>>>>>>>> I
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>           suggest to
> >>>>>>>>>>>>>>>>>>>           > no
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
> >>>>> overhead.
> >>>>>>> If
> >>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > problem in
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add name
> >>>>> spacing
> >>>>>>>>> later
> >>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the design
> it
> >> the
> >>>>>>> type
> >>>>>>>>> of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           result KTable:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable =
> >>>>>>>>>>>> table1.join(table2,...);
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the `joinedTable` has
> >> the
> >>>>>>> same
> >>>>>>>>> key
> >>>>>>>>>>>> as the
> >>>>>>>>>>>>>>>>>>>           left input
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work because
> if
> >>>>> table2
> >>>>>>>>>>>> contains
> >>>>>>>>>>>>>>>>>>>           multiple rows
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in table1
> (what is
> >>>>> the
> >>>>>>> main
> >>>>>>>>>>>> purpose
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>           > foreign
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table would only
> >>>>>>> contain a
> >>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>           join result,
> >>>>>>>>>>>>>>>>>>>           > but
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream: <a,(A,1)>,
> >>>>> <b,(A,2)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a foreign key
> to
> >>>>>>> table1
> >>>>>>>>> key
> >>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>           "A" joins).
> >>>>>>>>>>>>>>>>>>>           > If
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key as key
> of
> >>>>>>> table1,
> >>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>           implies that the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A,
> join(X,1)>
> >> or
> >>>>> <A,
> >>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>           but not
> >>>>>>>>>>>>>>>>>>>           > both.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same key,
> >>>>> whatever
> >>>>>>>>> result
> >>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>           we emit
> >>>>>>>>>>>>>>>>>>>           > later,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous result.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan
> originally
> >>>>>>> proposed
> >>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>           > combination
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the input
> tables
> >>>>> as
> >>>>>>> key
> >>>>>>>>> of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           output table.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the output table
> >>>>> unique
> >>>>>>> and
> >>>>>>>>> we
> >>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>           store both in
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a, join(X,1)>,
> >> <A-b,
> >>>>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak
> >> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could be
> >>>>> disregarded.
> >>>>>>> The
> >>>>>>>>>>>> decision
> >>>>>>>>>>>>>>>>>>>           about the
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of the
> >> aggregated
> >>>>>>> map.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps would be
> >>>>>>> unpacked
> >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>           forwarded. 0
> >>>>>>>>>>>>>>>>>>>           > element
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as delete. Any
> >>>>> other
> >>>>>>> count
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in "waiting for
> >>>>> correct
> >>>>>>>>>>>> deletes to
> >>>>>>>>>>>>>>>>>>>           > arrive"-state.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam
> Bellemare
> >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could replace
> >> the
> >>>>>>> second
> >>>>>>>>>>>>>>>>>>>           repartition store
> >>>>>>>>>>>>>>>>>>>           > and
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a groupBy
> and
> >>>>>>> reduce.
> >>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>           it looks
> >>>>>>>>>>>>>>>>>>>           > like
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the
> highwater
> >>>>> value
> >>>>>>>>> within
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           materialized
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of
> out-of-order
> >>>>>>> records
> >>>>>>>>>>>> (assuming
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This in
> effect
> >> is
> >>>>>>> the
> >>>>>>>>> same
> >>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>           design I
> >>>>>>>>>>>>>>>>>>>           > have
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables merged
> >>>>> together.
> >>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

On 10.12.2018 07:42, Guozhang Wang wrote:
> Hello Adam / Jan / John,
>
> Sorry for being late on this thread! I've finally got some time this
> weekend to cleanup a load of tasks on my queue (actually I've also realized
> there are a bunch of other things I need to enqueue while cleaning them up
> --- sth I need to improve on my side). So here are my thoughts:
>
> Regarding the APIs: I like the current written API in the KIP. More
> generally I'd prefer to keep the 1) one-to-many join functionalities as
> well as 2) other join types than inner as separate KIPs since 1) may worth
> a general API refactoring that can benefit not only foreignkey joins but
> collocate joins as well (e.g. an extended proposal of
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup),
> and I'm not sure if other join types would actually be needed (maybe left
> join still makes sense), so it's better to wait-for-people-to-ask-and-add
> than add-sth-that-no-one-uses.
>
> Regarding whether we enforce step 3) / 4) v.s. introducing a
> KScatteredTable for users to inject their own optimization: I'd prefer to
> do the current option as-is, and my main rationale is for optimization
> rooms inside the Streams internals and the API succinctness. For advanced
> users who may indeed prefer KScatteredTable and do their own optimization,
> while it is too much of the work to use Processor API directly, I think we
> can still extend the current API to support it in the future if it becomes
> necessary.

no internal optimization potential. it's a myth

¯\_(ツ)_/¯

:-)

>
> Another note about step 4) resolving out-of-ordering data, as I mentioned
> before I think with KIP-258 (embedded timestamp with key-value store) we
> can actually make this step simpler than the current proposal. In fact, we
> can just keep a single final-result store with timestamps and reject values
> that have a smaller timestamp, is that right?

Which is the correct output should at least be decided on the offset of 
the original message.

>
>
> That's all I have in mind now. Again, great appreciation to Adam to make
> such HUGE progress on this KIP!
>
>
> Guozhang
>
> On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> If they don't find the time:
>> They usually take the opposite path from me :D
>> so the answer would be clear.
>>
>> hence my suggestion to vote.
>>
>>
>> On 04.12.2018 21:06, Adam Bellemare wrote:
>>> Hi Guozhang and Matthias
>>>
>>> I know both of you are quite busy, but we've gotten this KIP to a point
>>> where we need more guidance on the API (perhaps a bit of a tie-breaker,
>> if
>>> you will). If you have anyone else you may think should look at this,
>>> please tag them accordingly.
>>>
>>> The scenario is as such:
>>>
>>> Current Option:
>>> API:
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
>>> 1) Rekey the data to CombinedKey, and shuffles it to the partition with
>> the
>>> foreignKey (repartition 1)
>>> 2) Join the data
>>> 3) Shuffle the data back to the original node (repartition 2)
>>> 4) Resolve out-of-order arrival / race condition due to foreign-key
>> changes.
>>>
>>> Alternate Option:
>>> Perform #1 and #2 above, and return a KScatteredTable.
>>> - It would be keyed on a wrapped key function: <CombinedKey<KO, K>, VR>
>> (KO
>>> = Other Table Key, K = This Table Key, VR = Joined Result)
>>> - KScatteredTable.resolve() would perform #3 and #4 but otherwise a user
>>> would be able to perform additional functions directly from the
>>> KScatteredTable (TBD - currently out of scope).
>>> - John's analysis 2-emails up is accurate as to the tradeoffs.
>>>
>>> Current Option is coded as-is. Alternate option is possible, but will
>>> require for implementation details to be made in the API and some
>> exposure
>>> of new data structures into the API (ie: CombinedKey).
>>>
>>> I appreciate any insight into this.
>>>
>>> Thanks.
>>>
>>> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <ad...@gmail.com>
>>> wrote:
>>>
>>>> Hi John
>>>>
>>>> Thanks for your feedback and assistance. I think your summary is
>> accurate
>>>> from my perspective. Additionally, I would like to add that there is a
>> risk
>>>> of inconsistent final states without performing the resolution. This is
>> a
>>>> major concern for me as most of the data I have dealt with is produced
>> by
>>>> relational databases. We have seen a number of cases where a user in the
>>>> Rails UI has modified the field (foreign key), realized they made a
>>>> mistake, and then updated the field again with a new key. The events are
>>>> propagated out as they are produced, and as such we have had real-world
>>>> cases where these inconsistencies were propagated downstream as the
>> final
>>>> values due to the race conditions in the fanout of the data.
>>>>
>>>> This solution that I propose values correctness of the final result over
>>>> other factors.
>>>>
>>>> We could always move this function over to using a KScatteredTable
>>>> implementation in the future, and simply deprecate it this join API in
>>>> time. I think I would like to hear more from some of the other major
>>>> committers on which course of action they would think is best before any
>>>> more coding is done.
>>>>
>>>> Thanks again
>>>>
>>>> Adam
>>>>
>>>>
>>>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io> wrote:
>>>>
>>>>> Hi Jan and Adam,
>>>>>
>>>>> Wow, thanks for doing that test, Adam. Those results are encouraging.
>>>>>
>>>>> Thanks for your performance experience as well, Jan. I agree that
>> avoiding
>>>>> unnecessary join outputs is especially important when the fan-out is so
>>>>> high. I suppose this could also be built into the implementation we're
>>>>> discussing, but it wouldn't have to be specified in the KIP (since
>> it's an
>>>>> API-transparent optimization).
>>>>>
>>>>> As far as whether or not to re-repartition the data, I didn't bring it
>> up
>>>>> because it sounded like the two of you agreed to leave the KIP as-is,
>>>>> despite the disagreement.
>>>>>
>>>>> If you want my opinion, I feel like both approaches are reasonable.
>>>>> It sounds like Jan values more the potential for developers to optimize
>>>>> their topologies to re-use the intermediate nodes, whereas Adam places
>>>>> more
>>>>> value on having a single operator that people can use without extra
>> steps
>>>>> at the end.
>>>>>
>>>>> Personally, although I do find it exceptionally annoying when a
>> framework
>>>>> gets in my way when I'm trying to optimize something, it seems better
>> to
>>>>> go
>>>>> for a single operation.
>>>>> * Encapsulating the internal transitions gives us significant latitude
>> in
>>>>> the implementation (for example, joining only at the end, not in the
>>>>> middle
>>>>> to avoid extra data copying and out-of-order resolution; how we
>> represent
>>>>> the first repartition keys (combined keys vs. value vectors), etc.).
>> If we
>>>>> publish something like a KScatteredTable with the right-partitioned
>> joined
>>>>> data, then the API pretty much locks in the implementation as well.
>>>>> * The API seems simpler to understand and use. I do mean "seems"; if
>>>>> anyone
>>>>> wants to make the case that KScatteredTable is actually simpler, I
>> think
>>>>> hypothetical usage code would help. From a relational algebra
>> perspective,
>>>>> it seems like KTable.join(KTable) should produce a new KTable in all
>>>>> cases.
>>>>> * That said, there might still be room in the API for a different
>>>>> operation
>>>>> like what Jan has proposed to scatter a KTable, and then do things like
>>>>> join, re-group, etc from there... I'm not sure; I haven't thought
>> through
>>>>> all the consequences yet.
>>>>>
>>>>> This is all just my opinion after thinking over the discussion so
>> far...
>>>>> -John
>>>>>
>>>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Updated the PR to take into account John's feedback.
>>>>>>
>>>>>> I did some preliminary testing for the performance of the prefixScan.
>> I
>>>>>> have attached the file, but I will also include the text in the body
>>>>> here
>>>>>> for archival purposes (I am not sure what happens to attached files).
>> I
>>>>>> also updated the PR and the KIP accordingly.
>>>>>>
>>>>>> Summary: It scales exceptionally well for scanning large values of
>>>>>> records. As Jan mentioned previously, the real issue would be more
>>>>> around
>>>>>> processing the resulting records after obtaining them. For instance,
>> it
>>>>>> takes approximately ~80-120 mS to flush the buffer and a further
>>>>> ~35-85mS
>>>>>> to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
>>>>>> through the records just to generate a simple count takes ~ 40 times
>>>>> longer
>>>>>> than the flush + scan combined.
>>>>>>
>>>>>>
>>>>>
>> ============================================================================================
>>>>>> Setup:
>>>>>>
>>>>>>
>>>>>
>> ============================================================================================
>>>>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m,
>> Xms512m)
>>>>>> CPU: i7 2.2 Ghz.
>>>>>>
>>>>>> Note: I am using a slightly-modified, directly-accessible Kafka
>> Streams
>>>>>> RocksDB
>>>>>> implementation (RocksDB.java, basically just avoiding the
>>>>>> ProcessorContext).
>>>>>> There are no modifications to the default RocksDB values provided in
>> the
>>>>>> 2.1/trunk release.
>>>>>>
>>>>>>
>>>>>> keysize = 128 bytes
>>>>>> valsize = 512 bytes
>>>>>>
>>>>>> Step 1:
>>>>>> Write X positive matching events: (key = prefix + left-padded
>>>>>> auto-incrementing integer)
>>>>>> Step 2:
>>>>>> Write 10X negative matching events (key = left-padded
>> auto-incrementing
>>>>>> integer)
>>>>>> Step 3:
>>>>>> Perform flush
>>>>>> Step 4:
>>>>>> Perform prefixScan
>>>>>> Step 5:
>>>>>> Iterate through return Iterator and validate the count of expected
>>>>> events.
>>>>>>
>>>>>>
>>>>>>
>>>>>
>> ============================================================================================
>>>>>> Results:
>>>>>>
>>>>>>
>>>>>
>> ============================================================================================
>>>>>> X = 1k (11k events total)
>>>>>> Flush Time = 39 mS
>>>>>> Scan Time = 7 mS
>>>>>> 6.9 MB disk
>>>>>>
>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>> X = 10k (110k events total)
>>>>>> Flush Time = 45 mS
>>>>>> Scan Time = 8 mS
>>>>>> 127 MB
>>>>>>
>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>> X = 100k (1.1M events total)
>>>>>> Test1:
>>>>>> Flush Time = 60 mS
>>>>>> Scan Time = 12 mS
>>>>>> 678 MB
>>>>>>
>>>>>> Test2:
>>>>>> Flush Time = 45 mS
>>>>>> Scan Time = 7 mS
>>>>>> 576 MB
>>>>>>
>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>> X = 1MB (11M events total)
>>>>>> Test1:
>>>>>> Flush Time = 52 mS
>>>>>> Scan Time = 19 mS
>>>>>> 7.2 GB
>>>>>>
>>>>>> Test2:
>>>>>> Flush Time = 84 mS
>>>>>> Scan Time = 34 mS
>>>>>> 9.1 GB
>>>>>>
>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>> X = 2.5M (27.5M events total)
>>>>>> Test1:
>>>>>> Flush Time = 82 mS
>>>>>> Scan Time = 63 mS
>>>>>> 17GB - 276 sst files
>>>>>>
>>>>>> Test2:
>>>>>> Flush Time = 116 mS
>>>>>> Scan Time = 35 mS
>>>>>> 23GB - 361 sst files
>>>>>>
>>>>>> Test3:
>>>>>> Flush Time = 103 mS
>>>>>> Scan Time = 82 mS
>>>>>> 19 GB - 300 sst files
>>>>>>
>>>>>>
>>>>>
>> --------------------------------------------------------------------------------------------
>>>>>>
>>>>>> I had to limit my testing on my laptop to X = 2.5M events. I tried to
>> go
>>>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+ range
>>>>> and my
>>>>>> laptop ran out of disk. More extensive testing could be done but I
>>>>> suspect
>>>>>> that it would be in line with what we're seeing in the results above.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> At this point in time, I think the only major discussion point is
>> really
>>>>>> around what Jan and I have disagreed on: repartitioning back +
>> resolving
>>>>>> potential out of order issues or leaving that up to the client to
>>>>> handle.
>>>>>>
>>>>>>
>>>>>> Thanks folks,
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Jan.Filipiak@trivago.com
>>>
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 29.11.2018 15:14, John Roesler wrote:
>>>>>>>> Hi all,
>>>>>>>>
>>>>>>>> Sorry that this discussion petered out... I think the 2.1 release
>>>>>>> caused an
>>>>>>>> extended distraction that pushed it off everyone's radar (which was
>>>>>>>> precisely Adam's concern). Personally, I've also had some extend
>>>>>>>> distractions of my own that kept (and continue to keep) me
>>>>> preoccupied.
>>>>>>>>
>>>>>>>> However, calling for a vote did wake me up, so I guess Jan was on
>> the
>>>>>>> right
>>>>>>>> track!
>>>>>>>>
>>>>>>>> I've gone back and reviewed the whole KIP document and the prior
>>>>>>>> discussion, and I'd like to offer a few thoughts:
>>>>>>>>
>>>>>>>> API Thoughts:
>>>>>>>>
>>>>>>>> 1. If I read the KIP right, you are proposing a many-to-one join.
>>>>> Could
>>>>>>> we
>>>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the design
>>>>>>> around
>>>>>>>> and make it a oneToManyJoin?
>>>>>>>>
>>>>>>>> The proposed name "joinOnForeignKey" disguises the join type, and it
>>>>>>> seems
>>>>>>>> like it might trick some people into using it for a one-to-one join.
>>>>>>> This
>>>>>>>> would work, of course, but it would be super inefficient compared to
>>>>> a
>>>>>>>> simple rekey-and-join.
>>>>>>>>
>>>>>>>> 2. I might have missed it, but I don't think it's specified whether
>>>>>>> it's an
>>>>>>>> inner, outer, or left join. I'm guessing an outer join, as
>>>>> (neglecting
>>>>>>> IQ),
>>>>>>>> the rest can be achieved by filtering or by handling it in the
>>>>>>> ValueJoiner.
>>>>>>>>
>>>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
>>>>>>>> 3a. Regarding Serialized: There are a few different paradigms in
>>>>> play in
>>>>>>>> the Streams API, so it's confusing, but instead of three Serialized
>>>>>>> args, I
>>>>>>>> think it would be better to have one that allows (optionally)
>> setting
>>>>>>> the 4
>>>>>>>> incoming serdes. The result serde is defined by the Materialized.
>> The
>>>>>>>> incoming serdes can be optional because they might already be
>>>>> available
>>>>>>> on
>>>>>>>> the source KTables, or the default serdes from the config might be
>>>>>>>> applicable.
>>>>>>>>
>>>>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't allow
>>>>>>> setting
>>>>>>>> one, and it seems like it might actually be harmful, since the rekey
>>>>>>>> operation needs to produce results that are co-partitioned with the
>>>>>>> "other"
>>>>>>>> KTable.
>>>>>>>>
>>>>>>>> 4. I'm fine with the "reserved word" header, but I didn't actually
>>>>>>> follow
>>>>>>>> what Matthias meant about namespacing requiring "deserializing" the
>>>>>>> record
>>>>>>>> header. The headers are already Strings, so I don't think that
>>>>>>>> deserialization is required. If we applied the namespace at source
>>>>> nodes
>>>>>>>> and stripped it at sink nodes, this would be practically no
>> overhead.
>>>>>>> The
>>>>>>>> advantage of the namespace idea is that no public API change wrt
>>>>> headers
>>>>>>>> needs to happen, and no restrictions need to be placed on users'
>>>>>>> headers.
>>>>>>>>
>>>>>>>> (Although I'm wondering if we can get away without the header at
>>>>> all...
>>>>>>>> stay tuned)
>>>>>>>>
>>>>>>>> 5. I also didn't follow the discussion about the HWM table growing
>>>>>>> without
>>>>>>>> bound. As I read it, the HWM table is effectively implementing OCC
>> to
>>>>>>>> resolve the problem you noted with disordering when the rekey is
>>>>>>>> reversed... particularly notable when the FK changes. As such, it
>>>>> only
>>>>>>>> needs to track the most recent "version" (the offset in the source
>>>>>>>> partition) of each key. Therefore, it should have the same number of
>>>>>>> keys
>>>>>>>> as the source table at all times.
>>>>>>>>
>>>>>>>> I see that you are aware of KIP-258, which I think might be relevant
>>>>> in
>>>>>>> a
>>>>>>>> couple of ways. One: it's just about storing the timestamp in the
>>>>> state
>>>>>>>> store, but the ultimate idea is to effectively use the timestamp as
>>>>> an
>>>>>>> OCC
>>>>>>>> "version" to drop disordered updates. You wouldn't want to use the
>>>>>>>> timestamp for this operation, but if you were to use a similar
>>>>>>> mechanism to
>>>>>>>> store the source offset in the store alongside the re-keyed values,
>>>>> then
>>>>>>>> you could avoid a separate table.
>>>>>>>>
>>>>>>>> 6. You and Jan have been thinking about this for a long time, so
>> I've
>>>>>>>> probably missed something here, but I'm wondering if we can avoid
>> the
>>>>>>> HWM
>>>>>>>> tracking at all and resolve out-of-order during a final join
>>>>> instead...
>>>>>>>>
>>>>>>>> Let's say we're joining a left table (Integer K: Letter FK, (other
>>>>>>> data))
>>>>>>>> to a right table (Letter K: (some data)).
>>>>>>>>
>>>>>>>> Left table:
>>>>>>>> 1: (A, xyz)
>>>>>>>> 2: (B, asd)
>>>>>>>>
>>>>>>>> Right table:
>>>>>>>> A: EntityA
>>>>>>>> B: EntityB
>>>>>>>>
>>>>>>>> We could do a rekey as you proposed with a combined key, but not
>>>>>>>> propagating the value at all..
>>>>>>>> Rekey table:
>>>>>>>> A-1: (dummy value)
>>>>>>>> B-2: (dummy value)
>>>>>>>>
>>>>>>>> Which we then join with the right table to produce:
>>>>>>>> A-1: EntityA
>>>>>>>> B-2: EntityB
>>>>>>>>
>>>>>>>> Which gets rekeyed back:
>>>>>>>> 1: A, EntityA
>>>>>>>> 2: B, EntityB
>>>>>>>>
>>>>>>>> And finally we do the actual join:
>>>>>>>> Result table:
>>>>>>>> 1: ((A, xyz), EntityA)
>>>>>>>> 2: ((B, asd), EntityB)
>>>>>>>>
>>>>>>>> The thing is that in that last join, we have the opportunity to
>>>>> compare
>>>>>>> the
>>>>>>>> current FK in the left table with the incoming PK of the right
>>>>> table. If
>>>>>>>> they don't match, we just drop the event, since it must be outdated.
>>>>>>>>
>>>>>>>
>>>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets updated
>> to
>>>>>>> (1:
>>>>>>>> B, xyz), ultimately yielding a conundrum about whether the final
>>>>> state
>>>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm above,
>>>>> you
>>>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
>>>>>>>> EntityB)). It seems like this does give you enough information to
>>>>> make
>>>>>>> the
>>>>>>>> right choice, regardless of disordering.
>>>>>>>
>>>>>>> Will check Adams patch, but this should work. As mentioned often I am
>>>>>>> not convinced on partitioning back for the user automatically. I
>> think
>>>>>>> this is the real performance eater ;)
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> 7. Last thought... I'm a little concerned about the performance of
>>>>> the
>>>>>>>> range scans when records change in the right table. You've said that
>>>>>>> you've
>>>>>>>> been using the algorithm you presented in production for a while.
>> Can
>>>>>>> you
>>>>>>>> give us a sense of the performance characteristics you've observed?
>>>>>>>>
>>>>>>>
>>>>>>> Make it work, make it fast, make it beautiful. The topmost thing here
>>>>> is
>>>>>>> / was correctness. In practice I do not measure the performance of
>> the
>>>>>>> range scan. Usual cases I run this with is emitting 500k - 1kk rows
>>>>>>> on a left hand side change. The range scan is just the work you gotta
>>>>>>> do, also when you pack your data into different formats, usually the
>>>>>>> rocks performance is very tight to the size of the data and we can't
>>>>>>> really change that. It is more important for users to prevent useless
>>>>>>> updates to begin with. My left hand side is guarded to drop changes
>>>>> that
>>>>>>> are not going to change my join output.
>>>>>>>
>>>>>>> usually it's:
>>>>>>>
>>>>>>> drop unused fields and then don't forward if old.equals(new)
>>>>>>>
>>>>>>> regarding to the performance of creating an iterator for smaller
>>>>>>> fanouts, users can still just do a group by first then anyways.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> I could only think of one alternative, but I'm not sure if it's
>>>>> better
>>>>>>> or
>>>>>>>> worse... If the first re-key only needs to preserve the original
>> key,
>>>>>>> as I
>>>>>>>> proposed in #6, then we could store a vector of keys in the value:
>>>>>>>>
>>>>>>>> Left table:
>>>>>>>> 1: A,...
>>>>>>>> 2: B,...
>>>>>>>> 3: A,...
>>>>>>>>
>>>>>>>> Gets re-keyed:
>>>>>>>> A: [1, 3]
>>>>>>>> B: [2]
>>>>>>>>
>>>>>>>> Then, the rhs part of the join would only need a regular single-key
>>>>>>> lookup.
>>>>>>>> Of course we have to deal with the problem of large values, as
>>>>> there's
>>>>>>> no
>>>>>>>> bound on the number of lhs records that can reference rhs records.
>>>>>>> Offhand,
>>>>>>>> I'd say we could page the values, so when one row is past the
>>>>>>> threshold, we
>>>>>>>> append the key for the next page. Then in most cases, it would be a
>>>>>>> single
>>>>>>>> key lookup, but for large fan-out updates, it would be one per (max
>>>>>>> value
>>>>>>>> size)/(avg lhs key size).
>>>>>>>>
>>>>>>>> This seems more complex, though... Plus, I think there's some extra
>>>>>>>> tracking we'd need to do to know when to emit a retraction. For
>>>>> example,
>>>>>>>> when record 1 is deleted, the re-key table would just have (A: [3]).
>>>>>>> Some
>>>>>>>> kind of tombstone is needed so that the join result for 1 can also
>> be
>>>>>>>> retracted.
>>>>>>>>
>>>>>>>> That's all!
>>>>>>>>
>>>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry
>> the
>>>>>>>> discussion has been slow.
>>>>>>>> -John
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>>>>> Jan.Filipiak@trivago.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Id say you can just call the vote.
>>>>>>>>>
>>>>>>>>> that happens all the time, and if something comes up, it just goes
>>>>> back
>>>>>>>>> to discuss.
>>>>>>>>>
>>>>>>>>> would not expect to much attention with another another email in
>>>>> this
>>>>>>>>> thread.
>>>>>>>>>
>>>>>>>>> best Jan
>>>>>>>>>
>>>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>>>>>>>>> Hello Contributors
>>>>>>>>>>
>>>>>>>>>> I know that 2.1 is about to be released, but I do need to bump
>>>>> this to
>>>>>>>>> keep
>>>>>>>>>> visibility up. I am still intending to push this through once
>>>>>>> contributor
>>>>>>>>>> feedback is given.
>>>>>>>>>>
>>>>>>>>>> Main points that need addressing:
>>>>>>>>>> 1) Any way (or benefit) in structuring the current singular graph
>>>>> node
>>>>>>>>> into
>>>>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I am a
>>>>> bit
>>>>>>>>> fuzzy
>>>>>>>>>> on how the optimizations are supposed to work, so I would
>>>>> appreciate
>>>>>>> any
>>>>>>>>>> help on this aspect.
>>>>>>>>>>
>>>>>>>>>> 2) Overall strategy for joining + resolving. This thread has much
>>>>>>>>> discourse
>>>>>>>>>> between Jan and I between the current highwater mark proposal and
>> a
>>>>>>>>> groupBy
>>>>>>>>>> + reduce proposal. I am of the opinion that we need to strictly
>>>>> handle
>>>>>>>>> any
>>>>>>>>>> chance of out-of-order data and leave none of it up to the
>>>>> consumer.
>>>>>>> Any
>>>>>>>>>> comments or suggestions here would also help.
>>>>>>>>>>
>>>>>>>>>> 3) Anything else that you see that would prevent this from moving
>>>>> to a
>>>>>>>>> vote?
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>>
>>>>>>>>>> Adam
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Jan
>>>>>>>>>>>
>>>>>>>>>>> With the Stores.windowStoreBuilder and
>>>>> Stores.persistentWindowStore,
>>>>>>> you
>>>>>>>>>>> actually only need to specify the amount of segments you want and
>>>>> how
>>>>>>>>> large
>>>>>>>>>>> they are. To the best of my understanding, what happens is that
>>>>> the
>>>>>>>>>>> segments are automatically rolled over as new data with new
>>>>>>> timestamps
>>>>>>>>> are
>>>>>>>>>>> created. We use this exact functionality in some of the work done
>>>>>>>>>>> internally at my company. For reference, this is the hopping
>>>>> windowed
>>>>>>>>> store.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>>>>>>>>
>>>>>>>>>>> In the code that I have provided, there are going to be two 24h
>>>>>>>>> segments.
>>>>>>>>>>> When a record is put into the windowStore, it will be inserted at
>>>>>>> time
>>>>>>>>> T in
>>>>>>>>>>> both segments. The two segments will always overlap by 12h. As
>>>>> time
>>>>>>>>> goes on
>>>>>>>>>>> and new records are added (say at time T+12h+), the oldest
>> segment
>>>>>>> will
>>>>>>>>> be
>>>>>>>>>>> automatically deleted and a new segment created. The records are
>>>>> by
>>>>>>>>> default
>>>>>>>>>>> inserted with the context.timestamp(), such that it is the record
>>>>>>> time,
>>>>>>>>> not
>>>>>>>>>>> the clock time, which is used.
>>>>>>>>>>>
>>>>>>>>>>> To the best of my understanding, the timestamps are retained when
>>>>>>>>>>> restoring from the changelog.
>>>>>>>>>>>
>>>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
>>>>>>> segment-level,
>>>>>>>>>>> instead of at an individual record level.
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>>>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Will that work? I expected it to blow up with ClassCastException
>>>>> or
>>>>>>>>>>>> similar.
>>>>>>>>>>>>
>>>>>>>>>>>> You either would have to specify the window you fetch/put or
>>>>> iterate
>>>>>>>>>>>> across all windows the key was found in right?
>>>>>>>>>>>>
>>>>>>>>>>>> I just hope the window-store doesn't check stream-time under the
>>>>>>> hoods
>>>>>>>>>>>> that would be a questionable interface.
>>>>>>>>>>>>
>>>>>>>>>>>> If it does: did you see my comment on checking all the windows
>>>>>>> earlier?
>>>>>>>>>>>> that would be needed to actually give reasonable time gurantees.
>>>>>>>>>>>>
>>>>>>>>>>>> Best
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>
>>>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the state
>>>>>>> store,
>>>>>>>>>>>> not
>>>>>>>>>>>>> the ProcessorSupplier.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the information. This is indeed something that
>>>>> will be
>>>>>>>>>>>>>>> extremely
>>>>>>>>>>>>>>> useful for this KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>> Thanks for your explanations. That being said, I will not be
>>>>>>> moving
>>>>>>>>>>>> ahead
>>>>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution as
>> you
>>>>>>>>>>>> propose.
>>>>>>>>>>>>>>> That being said, if you wish to implement it yourself off of
>>>>> my
>>>>>>>>>>>> current PR
>>>>>>>>>>>>>>> and submit it as a competitive alternative, I would be more
>>>>> than
>>>>>>>>>>>> happy to
>>>>>>>>>>>>>>> help vet that as an alternate solution. As it stands right
>>>>> now,
>>>>>>> I do
>>>>>>>>>>>> not
>>>>>>>>>>>>>>> really have more time to invest into alternatives without
>>>>> there
>>>>>>>>> being
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> strong indication from the binding voters which they would
>>>>>>> prefer.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on the
>>>>> streams
>>>>>>>>> DSL
>>>>>>>>>>>> for
>>>>>>>>>>>>>> some time already, otherwise I would have pulled this KIP
>>>>> through
>>>>>>>>>>>> already.
>>>>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I will look at finishing up my PR with the windowed state
>>>>> store
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>> week or so, exercising it via tests, and then I will come
>> back
>>>>>>> for
>>>>>>>>>>>> final
>>>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the binding
>>>>>>> voters
>>>>>>>>>>>> could
>>>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
>>>>> according
>>>>>>> to
>>>>>>>>> the
>>>>>>>>>>>>>>> latest plan:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store. This
>>>>>>> could
>>>>>>>>> be
>>>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
>>>>> completed.
>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated
>>>>> in
>>>>>>> the
>>>>>>>>>>>> PR?
>>>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing something?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong
>> link,
>>>>>>> as it
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258
>>>>> we do
>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>> have "handling out-of-order data for source KTable" such
>> that
>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>> blindly apply the updates to the materialized store, i.e.
>>>>>>> following
>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>> ordering, we will reject updates that are older than the
>>>>> current
>>>>>>>>>>>> key's
>>>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>>>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
>>>>> the
>>>>>>>>> high
>>>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a window
>>>>>>> store),
>>>>>>>>> I
>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>> another current on-going KIP may actually help:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value store
>>>>> (i.e.
>>>>>>> only
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
>>>>> described
>>>>>>> in
>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that
>>>>> we
>>>>>>> can
>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>> "reject" updates from the source topics if its timestamp is
>>>>>>>>> smaller
>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>> the current key's latest update timestamp. I think it is
>>>>> very
>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>> what you have in mind for high watermark based filtering,
>>>>> while
>>>>>>>>> you
>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>> need to make sure that the timestamps of the joining
>> records
>>>>>>> are
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> correctly
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> inherited though the whole topology to the final stage.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
>>>>>>> non-windowed
>>>>>>>>>>>> KTables
>>>>>>>>>>>>>>>>> only, but for windowed KTables we do not really have a good
>>>>>>>>> support
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> their joins anyways (
>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
>>>>> non-key
>>>>>>>>> joins
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
>> endlessly
>>>>>>> based
>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of (<this
>>>>> table
>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> key>,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
>>>>>>> differentiate
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal would be
>>>>> to
>>>>>>>>>>>> replace
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would allow
>>>>> the
>>>>>>>>> same
>>>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This should
>>>>> allow
>>>>>>> for
>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
>>>>>>> customizable
>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
>>>>>>> minutes
>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> window,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the trick
>> here.
>>>>>>> Even
>>>>>>>>>>>> if I
>>>>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
>>>>> optional
>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a little bit
>>>>>>>>>>>> sceptical
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> how to determine the window. So esentially one could run
>>>>> into
>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the rapid change happens near a window border. I will check
>>>>> you
>>>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we could
>>>>> still
>>>>>>>>> check
>>>>>>>>>>>>>>>>>> _all_
>>>>>>>>>>>>>>>>>> windows on read with not to bad performance impact I
>> guess.
>>>>>>> Will
>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
>>>>> wouldn't
>>>>>>> not
>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>>>>>>>>> timestamp(B).
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> we can't expect that.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks for the
>>>>>>>>>>>> diagram, it
>>>>>>>>>>>>>>>>>>> did really help. You are correct that I do not have the
>>>>>>> original
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> key available, and I can see that if it was available then
>>>>> you
>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That being
>>>>> said,
>>>>>>> I
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> encourage
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity for
>>>>>>> everyone
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> else.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But
>>>>> I
>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original
>> primary
>>>>>>> key,
>>>>>>>>> We
>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
>> basically
>>>>>>> not
>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
>>>>> original
>>>>>>> DSL
>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my end.
>>>>> Will
>>>>>>>>>>>> finish
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay
>> inside
>>>>>>> the
>>>>>>>>>>>> State
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
>>>>> propagated?
>>>>>>>>> Isn't
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state store?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
>>>>>>> return
>>>>>>>>>>>> `null`
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is going to
>>>>> be
>>>>>>> a
>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store directly
>>>>> for
>>>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
>>>>> store,
>>>>>>>>>>>> satisfying
>>>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
>>>>> Windowed
>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> keeping the values, so for the next statefull operation we
>>>>>>> would
>>>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the window
>>>>>>> store
>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the values then.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group by
>>>>> before
>>>>>>>>>>>>>>>>>> repartitioning to the original primary key i think it
>> would
>>>>>>> help
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> big time in building efficient apps. Given the original
>>>>> primary
>>>>>>>>> key
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> understand that we do not have a solid foundation to build
>>>>> on.
>>>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
>>>>>>> unfortunate. I
>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>> understand the decision goes like that. I do not think its
>>>>> a
>>>>>>> good
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> decision.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>>>>>>> dumbreprajakta311@gmail.com
>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           please remove me from this group
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > Hi Adam,
>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>           > give me some time, will make such a chart.
>> last
>>>>>>> time i
>>>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>>>           get along
>>>>>>>>>>>>>>>>>>>           > well with giphy and ruined all your charts.
>>>>>>>>>>>>>>>>>>>           > Hopefully i can get it done today
>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>           > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>           > > Hi Jan
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > > I have included a diagram of what I
>> attempted
>>>>> on
>>>>>>> the
>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>>>>>>>>>
>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>>>>>>>>           <
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>>>>>>>>>
>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > > I attempted this back at the start of my own
>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>           this
>>>>>>>>>>>>>>>>>>>           > > solution, and since I could not get it to
>>>>> work I
>>>>>>> have
>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>           discarded the
>>>>>>>>>>>>>>>>>>>           > > code. At this point in time, if you wish to
>>>>>>> continue
>>>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>>>           for your
>>>>>>>>>>>>>>>>>>>           > > groupBy solution, I ask that you please
>>>>> create a
>>>>>>>>>>>> diagram on
>>>>>>>>>>>>>>>>>>>           the KIP
>>>>>>>>>>>>>>>>>>>           > > carefully explaining your solution. Please
>>>>> feel
>>>>>>> free
>>>>>>>>> to
>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>           the image I
>>>>>>>>>>>>>>>>>>>           > > just posted as a starting point. I am having
>>>>>>> trouble
>>>>>>>>>>>>>>>>>>>           understanding your
>>>>>>>>>>>>>>>>>>>           > > explanations but I think that a carefully
>>>>>>> constructed
>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>>>           will clear
>>>>>>>>>>>>>>>>>>>           > up
>>>>>>>>>>>>>>>>>>>           > > any misunderstandings. Alternately, please
>>>>> post a
>>>>>>>>>>>>>>>>>>>           comprehensive PR with
>>>>>>>>>>>>>>>>>>>           > > your solution. I can only guess at what you
>>>>>>> mean, and
>>>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>>>           value my
>>>>>>>>>>>>>>>>>>>           > own
>>>>>>>>>>>>>>>>>>>           > > time as much as you value yours, I believe
>> it
>>>>> is
>>>>>>> your
>>>>>>>>>>>>>>>>>>>           responsibility to
>>>>>>>>>>>>>>>>>>>           > > provide an implementation instead of me
>>>>> trying to
>>>>>>>>> guess.
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > > Adam
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > > wrote:
>>>>>>>>>>>>>>>>>>>           > >
>>>>>>>>>>>>>>>>>>>           > >> Hi James,
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >> nice to see you beeing interested. kafka
>>>>>>> streams at
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>           point supports
>>>>>>>>>>>>>>>>>>>           > >> all sorts of joins as long as both streams
>>>>> have
>>>>>>> the
>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>           > >> Adam is currently implementing a join
>> where a
>>>>>>> KTable
>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>           KTable can
>>>>>>>>>>>>>>>>>>>           > have
>>>>>>>>>>>>>>>>>>>           > >> a one to many relation ship (1:n). We
>> exploit
>>>>>>> that
>>>>>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > >> datastore that keeps data sorted (At least
>>>>>>> exposes an
>>>>>>>>>>>> API to
>>>>>>>>>>>>>>>>>>>           access the
>>>>>>>>>>>>>>>>>>>           > >> stored data in a sorted fashion).
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >> I think the technical caveats are well
>>>>>>> understood
>>>>>>>>> now
>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > basically
>>>>>>>>>>>>>>>>>>>           > >> down to philosophy and API Design ( when
>> Adam
>>>>>>> sees
>>>>>>>>> my
>>>>>>>>>>>> newest
>>>>>>>>>>>>>>>>>>>           message).
>>>>>>>>>>>>>>>>>>>           > >> I have a lengthy track record of loosing
>>>>> those
>>>>>>> kinda
>>>>>>>>>>>>>>>>>>>           arguments within
>>>>>>>>>>>>>>>>>>>           > the
>>>>>>>>>>>>>>>>>>>           > >> streams community and I have no clue why.
>> So
>>>>> I
>>>>>>>>>>>> literally
>>>>>>>>>>>>>>>>>>>           can't wait for
>>>>>>>>>>>>>>>>>>>           > you
>>>>>>>>>>>>>>>>>>>           > >> to churn through this thread and give you
>>>>>>> opinion on
>>>>>>>>>>>> how we
>>>>>>>>>>>>>>>>>>>           should
>>>>>>>>>>>>>>>>>>>           > design
>>>>>>>>>>>>>>>>>>>           > >> the return type of the oneToManyJoin and
>> how
>>>>>>> many
>>>>>>>>>>>> power we
>>>>>>>>>>>>>>>>>>>           want to give
>>>>>>>>>>>>>>>>>>>           > to
>>>>>>>>>>>>>>>>>>>           > >> the user vs "simplicity" (where simplicity
>>>>> isn't
>>>>>>>>>>>> really that
>>>>>>>>>>>>>>>>>>>           as users
>>>>>>>>>>>>>>>>>>>           > still
>>>>>>>>>>>>>>>>>>>           > >> need to understand it I argue)
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >> waiting for you to join in on the
>> discussion
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >> Best Jan
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>>>>>>>>>>>>>>           > >>
>>>>>>>>>>>>>>>>>>>           > >>> I am new to this group and I found this
>>>>> subject
>>>>>>>>>>>>>>>>>>>           interesting.  Sounds
>>>>>>>>>>>>>>>>>>>           > like
>>>>>>>>>>>>>>>>>>>           > >>> you guys want to implement a join table of
>>>>> two
>>>>>>>>>>>> streams? Is
>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>           > somewhere
>>>>>>>>>>>>>>>>>>>           > >>> I can see the original requirement or
>>>>> proposal?
>>>>>>>>>>>>>>>>>>>           > >>>
>>>>>>>>>>>>>>>>>>>           > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>>>>>>>>>>>>>>           <Jan.Filipiak@trivago.com <mailto:
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>> On 05.09.2018 22:17, Adam Bellemare
>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> I'm currently testing using a Windowed
>>>>> Store
>>>>>>> to
>>>>>>>>>>>> store the
>>>>>>>>>>>>>>>>>>>           highwater
>>>>>>>>>>>>>>>>>>>           > >>>>> mark.
>>>>>>>>>>>>>>>>>>>           > >>>>> By all indications this should work
>> fine,
>>>>>>> with
>>>>>>>>> the
>>>>>>>>>>>> caveat
>>>>>>>>>>>>>>>>>>>           being that
>>>>>>>>>>>>>>>>>>>           > it
>>>>>>>>>>>>>>>>>>>           > >>>>> can
>>>>>>>>>>>>>>>>>>>           > >>>>> only resolve out-of-order arrival for up
>>>>> to
>>>>>>> the
>>>>>>>>>>>> size of
>>>>>>>>>>>>>>>>>>>           the window
>>>>>>>>>>>>>>>>>>>           > (ie:
>>>>>>>>>>>>>>>>>>>           > >>>>> 24h, 72h, etc). This would remove the
>>>>>>> possibility
>>>>>>>>>>>> of it
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > unbounded
>>>>>>>>>>>>>>>>>>>           > >>>>> in
>>>>>>>>>>>>>>>>>>>           > >>>>> size.
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> With regards to Jan's suggestion, I
>>>>> believe
>>>>>>> this
>>>>>>>>> is
>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>           we will
>>>>>>>>>>>>>>>>>>>           > have
>>>>>>>>>>>>>>>>>>>           > >>>>> to
>>>>>>>>>>>>>>>>>>>           > >>>>> remain in disagreement. While I do not
>>>>>>> disagree
>>>>>>>>>>>> with your
>>>>>>>>>>>>>>>>>>>           statement
>>>>>>>>>>>>>>>>>>>           > >>>>> about
>>>>>>>>>>>>>>>>>>>           > >>>>> there likely to be additional joins done
>>>>> in a
>>>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>>>           workflow, I
>>>>>>>>>>>>>>>>>>>           > do
>>>>>>>>>>>>>>>>>>>           > >>>>> not
>>>>>>>>>>>>>>>>>>>           > >>>>> see how you can conclusively deal with
>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>           > >>>>> foreign-key
>>>>>>>>>>>>>>>>>>>           > >>>>> changes and subsequent joins. I have
>>>>>>> attempted
>>>>>>>>> what
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>           think you have
>>>>>>>>>>>>>>>>>>>           > >>>>> proposed (without a high-water, using
>>>>>>> groupBy and
>>>>>>>>>>>> reduce)
>>>>>>>>>>>>>>>>>>>           and found
>>>>>>>>>>>>>>>>>>>           > >>>>> that if
>>>>>>>>>>>>>>>>>>>           > >>>>> the foreign key changes too quickly, or
>>>>> the
>>>>>>> load
>>>>>>>>> on
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>           stream thread
>>>>>>>>>>>>>>>>>>>           > is
>>>>>>>>>>>>>>>>>>>           > >>>>> too
>>>>>>>>>>>>>>>>>>>           > >>>>> high, the joined messages will arrive
>>>>>>>>> out-of-order
>>>>>>>>>>>> and be
>>>>>>>>>>>>>>>>>>>           incorrectly
>>>>>>>>>>>>>>>>>>>           > >>>>> propagated, such that an intermediate
>>>>> event
>>>>>>> is
>>>>>>>>>>>>>>>>>>> represented
>>>>>>>>>>>>>>>>>>>           as the
>>>>>>>>>>>>>>>>>>>           > final
>>>>>>>>>>>>>>>>>>>           > >>>>> event.
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>> Can you shed some light on your groupBy
>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>           There must be
>>>>>>>>>>>>>>>>>>>           > >>>> some sort of flaw in it.
>>>>>>>>>>>>>>>>>>>           > >>>> I have a suspicion where it is, I would
>>>>> just
>>>>>>> like
>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>           confirm. The idea
>>>>>>>>>>>>>>>>>>>           > >>>> is bullet proof and it must be
>>>>>>>>>>>>>>>>>>>           > >>>> an implementation mess up. I would like
>> to
>>>>>>> clarify
>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>           we draw a
>>>>>>>>>>>>>>>>>>>           > >>>> conclusion.
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>    Repartitioning the scattered events
>>>>> back to
>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > >>>>> partitions is the only way I know how to
>>>>>>>>> conclusively
>>>>>>>>>>>> deal
>>>>>>>>>>>>>>>>>>>           with
>>>>>>>>>>>>>>>>>>>           > >>>>> out-of-order events in a given time
>> frame,
>>>>>>> and to
>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>           that the
>>>>>>>>>>>>>>>>>>>           > data
>>>>>>>>>>>>>>>>>>>           > >>>>> is
>>>>>>>>>>>>>>>>>>>           > >>>>> eventually consistent with the input
>>>>> events.
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> If you have some code to share that
>>>>>>> illustrates
>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>           approach, I
>>>>>>>>>>>>>>>>>>>           > would
>>>>>>>>>>>>>>>>>>>           > >>>>> be
>>>>>>>>>>>>>>>>>>>           > >>>>> very grateful as it would remove any
>>>>>>>>>>>> misunderstandings
>>>>>>>>>>>>>>>>>>>           that I may
>>>>>>>>>>>>>>>>>>>           > have.
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>> ah okay you were looking for my code. I
>>>>> don't
>>>>>>> have
>>>>>>>>>>>>>>>>>>>           something easily
>>>>>>>>>>>>>>>>>>>           > >>>> readable here as its bloated with
>>>>> OO-patterns.
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>> its anyhow trivial:
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>> @Override
>>>>>>>>>>>>>>>>>>>           > >>>>      public T apply(K aggKey, V value, T
>>>>>>>>> aggregate)
>>>>>>>>>>>>>>>>>>>           > >>>>      {
>>>>>>>>>>>>>>>>>>>           > >>>>          Map<U, V> currentStateAsMap =
>>>>>>>>>>>> asMap(aggregate);
>>>>>>>>>>>>>>>>>>> <<
>>>>>>>>>>>>>>>>>>>           imaginary
>>>>>>>>>>>>>>>>>>>           > >>>>          U toModifyKey =
>>>>> mapper.apply(value);
>>>>>>>>>>>>>>>>>>>           > >>>>              << this is the place where
>>>>> people
>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>           gonna have
>>>>>>>>>>>>>>>>>>>           > issues
>>>>>>>>>>>>>>>>>>>           > >>>> and why you probably couldn't do it. we
>>>>> would
>>>>>>> need
>>>>>>>>>>>> to find
>>>>>>>>>>>>>>>>>>>           a solution
>>>>>>>>>>>>>>>>>>>           > here.
>>>>>>>>>>>>>>>>>>>           > >>>> I didn't realize that yet.
>>>>>>>>>>>>>>>>>>>           > >>>>              << we propagate the field in
>>>>> the
>>>>>>>>>>>> joiner, so
>>>>>>>>>>>>>>>>>>>           that we can
>>>>>>>>>>>>>>>>>>>           > pick
>>>>>>>>>>>>>>>>>>>           > >>>> it up in an aggregate. Probably you have
>>>>> not
>>>>>>>>> thought
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>           this in your
>>>>>>>>>>>>>>>>>>>           > >>>> approach right?
>>>>>>>>>>>>>>>>>>>           > >>>>              << I am very open to find a
>>>>>>> generic
>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>           here. In my
>>>>>>>>>>>>>>>>>>>           > >>>> honest opinion this is broken in
>>>>>>>>> KTableImpl.GroupBy
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>           looses
>>>>>>>>>>>>>>>>>>>           > the keys
>>>>>>>>>>>>>>>>>>>           > >>>> and only maintains the aggregate key.
>>>>>>>>>>>>>>>>>>>           > >>>>              << I abstracted it away back
>>>>>>> then way
>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>           > thinking
>>>>>>>>>>>>>>>>>>>           > >>>> of oneToMany join. That is why I didn't
>>>>>>> realize
>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>           significance here.
>>>>>>>>>>>>>>>>>>>           > >>>>              << Opinions?
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>          for (V m : current)
>>>>>>>>>>>>>>>>>>>           > >>>>          {
>>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(mapper.apply(m),
>> m);
>>>>>>>>>>>>>>>>>>>           > >>>>          }
>>>>>>>>>>>>>>>>>>>           > >>>>          if (isAdder)
>>>>>>>>>>>>>>>>>>>           > >>>>          {
>>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.put(toModifyKey,
>> value);
>>>>>>>>>>>>>>>>>>>           > >>>>          }
>>>>>>>>>>>>>>>>>>>           > >>>>          else
>>>>>>>>>>>>>>>>>>>           > >>>>          {
>>>>>>>>>>>>>>>>>>>           > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>>>>>>>>           > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>>>>>>>>           > >>>>                  return null;
>>>>>>>>>>>>>>>>>>>           > >>>>              }
>>>>>>>>>>>>>>>>>>>           > >>>>          }
>>>>>>>>>>>>>>>>>>>           > >>>>          retrun
>>>>>>> asAggregateType(currentStateAsMap)
>>>>>>>>>>>>>>>>>>>           > >>>>      }
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>>
>>>>>>>>>>>>>>>>>>>           > >>>> Thanks,
>>>>>>>>>>>>>>>>>>>           > >>>>> Adam
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
>>>>> Filipiak
>>>>>>> <
>>>>>>>>>>>>>>>>>>>           > Jan.Filipiak@trivago.com <mailto:
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>> Thanks Adam for bringing Matthias to
>>>>> speed!
>>>>>>>>>>>>>>>>>>>           > >>>>>> about the differences. I think
>> re-keying
>>>>>>> back
>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>           optional at
>>>>>>>>>>>>>>>>>>>           > >>>>>> best.
>>>>>>>>>>>>>>>>>>>           > >>>>>> I would say we return a KScatteredTable
>>>>> with
>>>>>>>>>>>> reshuffle()
>>>>>>>>>>>>>>>>>>>           returning
>>>>>>>>>>>>>>>>>>>           > >>>>>> KTable<originalKey,Joined> to make the
>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>           repartitioning
>>>>>>>>>>>>>>>>>>>           > >>>>>> optional.
>>>>>>>>>>>>>>>>>>>           > >>>>>> I am also in a big favour of doing the
>>>>> out
>>>>>>> of
>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>           processing using
>>>>>>>>>>>>>>>>>>>           > >>>>>> group
>>>>>>>>>>>>>>>>>>>           > >>>>>> by instead high water mark tracking.
>>>>>>>>>>>>>>>>>>>           > >>>>>> Just because unbounded growth is just
>>>>> scary
>>>>>>> + It
>>>>>>>>>>>> saves
>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>           the header
>>>>>>>>>>>>>>>>>>>           > >>>>>> stuff.
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>> I think the abstraction of always
>>>>>>> repartitioning
>>>>>>>>>>>> back is
>>>>>>>>>>>>>>>>>>>           just not so
>>>>>>>>>>>>>>>>>>>           > >>>>>> strong. Like the work has been done
>>>>> before
>>>>>>> we
>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>           back and
>>>>>>>>>>>>>>>>>>>           > >>>>>> grouping
>>>>>>>>>>>>>>>>>>>           > >>>>>> by something else afterwards is really
>>>>>>> common.
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>> On 05.09.2018 13:49, Adam Bellemare
>>>>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>> Hi Matthias
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Thank you for your feedback, I do
>>>>>>> appreciate
>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> While name spacing would be possible,
>> it
>>>>>>> would
>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>           > deserialize
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a runtime
>>>>>>> overhead.
>>>>>>>>> I
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>           suggest to
>>>>>>>>>>>>>>>>>>>           > no
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
>>>>> overhead.
>>>>>>> If
>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > problem in
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add name
>>>>> spacing
>>>>>>>>> later
>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Agreed. I will go with using a
>> reserved
>>>>>>> string
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>           document it.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> My main concern about the design it
>> the
>>>>>>> type of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           result KTable:
>>>>>>>>>>>>>>>>>>>           > If
>>>>>>>>>>>>>>>>>>>           > >>>>>>> I
>>>>>>>>>>>>>>>>>>>           > >>>>>>> understood the proposal correctly,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> In your example, you have table1 and
>>>>> table2
>>>>>>>>>>>> swapped.
>>>>>>>>>>>>>>>>>>>           Here is how it
>>>>>>>>>>>>>>>>>>>           > >>>>>>> works
>>>>>>>>>>>>>>>>>>>           > >>>>>>> currently:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> 1) table1 has the records that contain
>>>>> the
>>>>>>>>>>>> foreign key
>>>>>>>>>>>>>>>>>>>           within their
>>>>>>>>>>>>>>>>>>>           > >>>>>>> value.
>>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>           > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> 2) A Value mapper is required to
>> extract
>>>>>>> the
>>>>>>>>>>>> foreign
>>>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 foreign key mapper: ( value =>
>>>>>>> value.fk
>>>>>>>>>>>>>>>>>>>           <http://value.fk> )
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> The mapper is applied to each element
>> in
>>>>>>>>> table1,
>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>           new combined
>>>>>>>>>>>>>>>>>>>           > >>>>>>> key is
>>>>>>>>>>>>>>>>>>>           > >>>>>>> made:
>>>>>>>>>>>>>>>>>>>           > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>,
>>>>> <A-b,
>>>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>>>           <B-c,
>>>>>>>>>>>>>>>>>>>           > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> 3) The rekeyed events are
>> copartitioned
>>>>>>> with
>>>>>>>>>>>> table2:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <A-a,
>>>>> (fk=A,bar=1)>,
>>>>>>> <A-b,
>>>>>>>>>>>>>>>>>>>           (fk=A,bar=2)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>>>>>>>>>>>>>>           > >>>>>>> RepartitionedTable1: <B-c,
>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> 4) From here, they can be joined
>>>>> together
>>>>>>>>> locally
>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>           applying the
>>>>>>>>>>>>>>>>>>>           > >>>>>>> joiner
>>>>>>>>>>>>>>>>>>>           > >>>>>>> function.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> At this point, Jan's design and my
>>>>> design
>>>>>>>>>>>> deviate. My
>>>>>>>>>>>>>>>>>>>           design goes
>>>>>>>>>>>>>>>>>>>           > on
>>>>>>>>>>>>>>>>>>>           > >>>>>>> to
>>>>>>>>>>>>>>>>>>>           > >>>>>>> repartition the data post-join and
>>>>> resolve
>>>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>>>           arrival of
>>>>>>>>>>>>>>>>>>>           > >>>>>>> records,
>>>>>>>>>>>>>>>>>>>           > >>>>>>> finally returning the data keyed just
>>>>> the
>>>>>>>>>>>> original key.
>>>>>>>>>>>>>>>>>>>           I do not
>>>>>>>>>>>>>>>>>>>           > >>>>>>> expose
>>>>>>>>>>>>>>>>>>>           > >>>>>>> the
>>>>>>>>>>>>>>>>>>>           > >>>>>>> CombinedKey or any of the internals
>>>>>>> outside of
>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           joinOnForeignKey
>>>>>>>>>>>>>>>>>>>           > >>>>>>> function. This does make for larger
>>>>>>> footprint,
>>>>>>>>>>>> but it
>>>>>>>>>>>>>>>>>>>           removes all
>>>>>>>>>>>>>>>>>>>           > >>>>>>> agency
>>>>>>>>>>>>>>>>>>>           > >>>>>>> for resolving out-of-order arrivals
>> and
>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>           CombinedKeys from
>>>>>>>>>>>>>>>>>>>           > the
>>>>>>>>>>>>>>>>>>>           > >>>>>>> user. I believe that this makes the
>>>>>>> function
>>>>>>>>> much
>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>           to use.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Let me know if this helps resolve your
>>>>>>>>> questions,
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>           please feel
>>>>>>>>>>>>>>>>>>>           > >>>>>>> free to
>>>>>>>>>>>>>>>>>>>           > >>>>>>> add anything else on your mind.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Adam
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
>>>>> Matthias J.
>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>           > >>>>>>> matthias@confluent.io <mailto:
>>>>>>>>>>>> matthias@confluent.io>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> I am just catching up on this
>> thread. I
>>>>>>> did
>>>>>>>>> not
>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>>>>           everything so
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> far,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> but want to share couple of initial
>>>>>>> thoughts:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Headers: I think there is a
>> fundamental
>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>           between header
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> usage
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> in this KIP and KP-258. For 258, we
>> add
>>>>>>>>> headers
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>           changelog topic
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> are owned by Kafka Streams and nobody
>>>>>>> else is
>>>>>>>>>>>> supposed
>>>>>>>>>>>>>>>>>>>           to write
>>>>>>>>>>>>>>>>>>>           > into
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> them. In fact, no user header are
>>>>> written
>>>>>>> into
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           changelog topic
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> thus, there are not conflicts.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Nevertheless, I don't see a big issue
>>>>> with
>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>           headers within
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Streams.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> As long as we document it, we can
>> have
>>>>>>> some
>>>>>>>>>>>> "reserved"
>>>>>>>>>>>>>>>>>>>           header keys
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> and
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> users are not allowed to use when
>>>>>>> processing
>>>>>>>>>>>> data with
>>>>>>>>>>>>>>>>>>>           Kafka
>>>>>>>>>>>>>>>>>>>           > Streams.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this should be ok.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> I think there is a safe way to avoid
>>>>>>>>> conflicts,
>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>           > headers
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> are
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> only needed in internal topics (I
>>>>> think):
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> For internal and changelog topics,
>> we
>>>>> can
>>>>>>>>>>>> namespace
>>>>>>>>>>>>>>>>>>>           all headers:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * user-defined headers are
>> namespaced
>>>>> as
>>>>>>>>>>>> "external."
>>>>>>>>>>>>>>>>>>> +
>>>>>>>>>>>>>>>>>>>           headerKey
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> * internal headers are namespaced as
>>>>>>>>>>>> "internal." +
>>>>>>>>>>>>>>>>>>>           headerKey
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> While name spacing would be
>> possible,
>>>>> it
>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > >>>>>>>> deserialize
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> user headers what implies a runtime
>>>>>>> overhead.
>>>>>>>>> I
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>           suggest to
>>>>>>>>>>>>>>>>>>>           > no
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> namespace for now to avoid the
>>>>> overhead.
>>>>>>> If
>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > problem in
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> the future, we can still add name
>>>>> spacing
>>>>>>>>> later
>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> My main concern about the design it
>> the
>>>>>>> type
>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           result KTable:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> If I
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> understood the proposal correctly,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> KTable<K1,V3> joinedTable =
>>>>>>>>>>>> table1.join(table2,...);
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> implies that the `joinedTable` has
>> the
>>>>>>> same
>>>>>>>>> key
>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>           left input
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> table.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> IMHO, this does not work because if
>>>>> table2
>>>>>>>>>>>> contains
>>>>>>>>>>>>>>>>>>>           multiple rows
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> that
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> join with a record in table1 (what is
>>>>> the
>>>>>>> main
>>>>>>>>>>>> purpose
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>           > foreign
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> key
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> join), the result table would only
>>>>>>> contain a
>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>           join result,
>>>>>>>>>>>>>>>>>>>           > but
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> not
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> multiple.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Example:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> table1 input stream: <A,X>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> table2 input stream: <a,(A,1)>,
>>>>> <b,(A,2)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> We use table2 value a foreign key to
>>>>>>> table1
>>>>>>>>> key
>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>           "A" joins).
>>>>>>>>>>>>>>>>>>>           > If
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> result key is the same key as key of
>>>>>>> table1,
>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>           implies that the
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> result can either be <A, join(X,1)>
>> or
>>>>> <A,
>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>           but not
>>>>>>>>>>>>>>>>>>>           > both.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Because the share the same key,
>>>>> whatever
>>>>>>>>> result
>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>           we emit
>>>>>>>>>>>>>>>>>>>           > later,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> overwrite the previous result.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> This is the reason why Jan originally
>>>>>>> proposed
>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>           > combination
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> of
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> both primary keys of the input tables
>>>>> as
>>>>>>> key
>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           output table.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> This
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> makes the keys of the output table
>>>>> unique
>>>>>>> and
>>>>>>>>> we
>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>           store both in
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> the
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> output table:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Result would be <A-a, join(X,1)>,
>> <A-b,
>>>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak
>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>> Just on remark here.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> The high-watermark could be
>>>>> disregarded.
>>>>>>> The
>>>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>>>           about the
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> depends on the size of the
>> aggregated
>>>>>>> map.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> Only 1 element long maps would be
>>>>>>> unpacked
>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>           forwarded. 0
>>>>>>>>>>>>>>>>>>>           > element
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> maps
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> would be published as delete. Any
>>>>> other
>>>>>>> count
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of map entries is in "waiting for
>>>>> correct
>>>>>>>>>>>> deletes to
>>>>>>>>>>>>>>>>>>>           > arrive"-state.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> It does look like I could replace
>> the
>>>>>>> second
>>>>>>>>>>>>>>>>>>>           repartition store
>>>>>>>>>>>>>>>>>>>           > and
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> highwater store with a groupBy and
>>>>>>> reduce.
>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>           it looks
>>>>>>>>>>>>>>>>>>>           > like
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> still need to store the highwater
>>>>> value
>>>>>>>>> within
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           materialized
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> compare the arrival of out-of-order
>>>>>>> records
>>>>>>>>>>>> (assuming
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           > >>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> THIS is correct...). This in effect
>> is
>>>>>>> the
>>>>>>>>> same
>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>           design I
>>>>>>>>>>>>>>>>>>>           > have
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>> just with the two tables merged
>>>>> together.
>>>>>>>>>>>>>>>>>>>           > >>>>>>>>>
>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Adam / Jan / John,

Sorry for being late on this thread! I've finally got some time this
weekend to cleanup a load of tasks on my queue (actually I've also realized
there are a bunch of other things I need to enqueue while cleaning them up
--- sth I need to improve on my side). So here are my thoughts:

Regarding the APIs: I like the current written API in the KIP. More
generally I'd prefer to keep the 1) one-to-many join functionalities as
well as 2) other join types than inner as separate KIPs since 1) may worth
a general API refactoring that can benefit not only foreignkey joins but
collocate joins as well (e.g. an extended proposal of
https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup),
and I'm not sure if other join types would actually be needed (maybe left
join still makes sense), so it's better to wait-for-people-to-ask-and-add
than add-sth-that-no-one-uses.

Regarding whether we enforce step 3) / 4) v.s. introducing a
KScatteredTable for users to inject their own optimization: I'd prefer to
do the current option as-is, and my main rationale is for optimization
rooms inside the Streams internals and the API succinctness. For advanced
users who may indeed prefer KScatteredTable and do their own optimization,
while it is too much of the work to use Processor API directly, I think we
can still extend the current API to support it in the future if it becomes
necessary.

Another note about step 4) resolving out-of-ordering data, as I mentioned
before I think with KIP-258 (embedded timestamp with key-value store) we
can actually make this step simpler than the current proposal. In fact, we
can just keep a single final-result store with timestamps and reject values
that have a smaller timestamp, is that right?


That's all I have in mind now. Again, great appreciation to Adam to make
such HUGE progress on this KIP!


Guozhang

On Wed, Dec 5, 2018 at 2:40 PM Jan Filipiak <Ja...@trivago.com>
wrote:

> If they don't find the time:
> They usually take the opposite path from me :D
> so the answer would be clear.
>
> hence my suggestion to vote.
>
>
> On 04.12.2018 21:06, Adam Bellemare wrote:
> > Hi Guozhang and Matthias
> >
> > I know both of you are quite busy, but we've gotten this KIP to a point
> > where we need more guidance on the API (perhaps a bit of a tie-breaker,
> if
> > you will). If you have anyone else you may think should look at this,
> > please tag them accordingly.
> >
> > The scenario is as such:
> >
> > Current Option:
> > API:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> > 1) Rekey the data to CombinedKey, and shuffles it to the partition with
> the
> > foreignKey (repartition 1)
> > 2) Join the data
> > 3) Shuffle the data back to the original node (repartition 2)
> > 4) Resolve out-of-order arrival / race condition due to foreign-key
> changes.
> >
> > Alternate Option:
> > Perform #1 and #2 above, and return a KScatteredTable.
> > - It would be keyed on a wrapped key function: <CombinedKey<KO, K>, VR>
> (KO
> > = Other Table Key, K = This Table Key, VR = Joined Result)
> > - KScatteredTable.resolve() would perform #3 and #4 but otherwise a user
> > would be able to perform additional functions directly from the
> > KScatteredTable (TBD - currently out of scope).
> > - John's analysis 2-emails up is accurate as to the tradeoffs.
> >
> > Current Option is coded as-is. Alternate option is possible, but will
> > require for implementation details to be made in the API and some
> exposure
> > of new data structures into the API (ie: CombinedKey).
> >
> > I appreciate any insight into this.
> >
> > Thanks.
> >
> > On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <ad...@gmail.com>
> > wrote:
> >
> >> Hi John
> >>
> >> Thanks for your feedback and assistance. I think your summary is
> accurate
> >> from my perspective. Additionally, I would like to add that there is a
> risk
> >> of inconsistent final states without performing the resolution. This is
> a
> >> major concern for me as most of the data I have dealt with is produced
> by
> >> relational databases. We have seen a number of cases where a user in the
> >> Rails UI has modified the field (foreign key), realized they made a
> >> mistake, and then updated the field again with a new key. The events are
> >> propagated out as they are produced, and as such we have had real-world
> >> cases where these inconsistencies were propagated downstream as the
> final
> >> values due to the race conditions in the fanout of the data.
> >>
> >> This solution that I propose values correctness of the final result over
> >> other factors.
> >>
> >> We could always move this function over to using a KScatteredTable
> >> implementation in the future, and simply deprecate it this join API in
> >> time. I think I would like to hear more from some of the other major
> >> committers on which course of action they would think is best before any
> >> more coding is done.
> >>
> >> Thanks again
> >>
> >> Adam
> >>
> >>
> >> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io> wrote:
> >>
> >>> Hi Jan and Adam,
> >>>
> >>> Wow, thanks for doing that test, Adam. Those results are encouraging.
> >>>
> >>> Thanks for your performance experience as well, Jan. I agree that
> avoiding
> >>> unnecessary join outputs is especially important when the fan-out is so
> >>> high. I suppose this could also be built into the implementation we're
> >>> discussing, but it wouldn't have to be specified in the KIP (since
> it's an
> >>> API-transparent optimization).
> >>>
> >>> As far as whether or not to re-repartition the data, I didn't bring it
> up
> >>> because it sounded like the two of you agreed to leave the KIP as-is,
> >>> despite the disagreement.
> >>>
> >>> If you want my opinion, I feel like both approaches are reasonable.
> >>> It sounds like Jan values more the potential for developers to optimize
> >>> their topologies to re-use the intermediate nodes, whereas Adam places
> >>> more
> >>> value on having a single operator that people can use without extra
> steps
> >>> at the end.
> >>>
> >>> Personally, although I do find it exceptionally annoying when a
> framework
> >>> gets in my way when I'm trying to optimize something, it seems better
> to
> >>> go
> >>> for a single operation.
> >>> * Encapsulating the internal transitions gives us significant latitude
> in
> >>> the implementation (for example, joining only at the end, not in the
> >>> middle
> >>> to avoid extra data copying and out-of-order resolution; how we
> represent
> >>> the first repartition keys (combined keys vs. value vectors), etc.).
> If we
> >>> publish something like a KScatteredTable with the right-partitioned
> joined
> >>> data, then the API pretty much locks in the implementation as well.
> >>> * The API seems simpler to understand and use. I do mean "seems"; if
> >>> anyone
> >>> wants to make the case that KScatteredTable is actually simpler, I
> think
> >>> hypothetical usage code would help. From a relational algebra
> perspective,
> >>> it seems like KTable.join(KTable) should produce a new KTable in all
> >>> cases.
> >>> * That said, there might still be room in the API for a different
> >>> operation
> >>> like what Jan has proposed to scatter a KTable, and then do things like
> >>> join, re-group, etc from there... I'm not sure; I haven't thought
> through
> >>> all the consequences yet.
> >>>
> >>> This is all just my opinion after thinking over the discussion so
> far...
> >>> -John
> >>>
> >>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <
> adam.bellemare@gmail.com>
> >>> wrote:
> >>>
> >>>> Updated the PR to take into account John's feedback.
> >>>>
> >>>> I did some preliminary testing for the performance of the prefixScan.
> I
> >>>> have attached the file, but I will also include the text in the body
> >>> here
> >>>> for archival purposes (I am not sure what happens to attached files).
> I
> >>>> also updated the PR and the KIP accordingly.
> >>>>
> >>>> Summary: It scales exceptionally well for scanning large values of
> >>>> records. As Jan mentioned previously, the real issue would be more
> >>> around
> >>>> processing the resulting records after obtaining them. For instance,
> it
> >>>> takes approximately ~80-120 mS to flush the buffer and a further
> >>> ~35-85mS
> >>>> to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
> >>>> through the records just to generate a simple count takes ~ 40 times
> >>> longer
> >>>> than the flush + scan combined.
> >>>>
> >>>>
> >>>
> ============================================================================================
> >>>> Setup:
> >>>>
> >>>>
> >>>
> ============================================================================================
> >>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m,
> Xms512m)
> >>>> CPU: i7 2.2 Ghz.
> >>>>
> >>>> Note: I am using a slightly-modified, directly-accessible Kafka
> Streams
> >>>> RocksDB
> >>>> implementation (RocksDB.java, basically just avoiding the
> >>>> ProcessorContext).
> >>>> There are no modifications to the default RocksDB values provided in
> the
> >>>> 2.1/trunk release.
> >>>>
> >>>>
> >>>> keysize = 128 bytes
> >>>> valsize = 512 bytes
> >>>>
> >>>> Step 1:
> >>>> Write X positive matching events: (key = prefix + left-padded
> >>>> auto-incrementing integer)
> >>>> Step 2:
> >>>> Write 10X negative matching events (key = left-padded
> auto-incrementing
> >>>> integer)
> >>>> Step 3:
> >>>> Perform flush
> >>>> Step 4:
> >>>> Perform prefixScan
> >>>> Step 5:
> >>>> Iterate through return Iterator and validate the count of expected
> >>> events.
> >>>>
> >>>>
> >>>>
> >>>
> ============================================================================================
> >>>> Results:
> >>>>
> >>>>
> >>>
> ============================================================================================
> >>>> X = 1k (11k events total)
> >>>> Flush Time = 39 mS
> >>>> Scan Time = 7 mS
> >>>> 6.9 MB disk
> >>>>
> >>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>> X = 10k (110k events total)
> >>>> Flush Time = 45 mS
> >>>> Scan Time = 8 mS
> >>>> 127 MB
> >>>>
> >>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>> X = 100k (1.1M events total)
> >>>> Test1:
> >>>> Flush Time = 60 mS
> >>>> Scan Time = 12 mS
> >>>> 678 MB
> >>>>
> >>>> Test2:
> >>>> Flush Time = 45 mS
> >>>> Scan Time = 7 mS
> >>>> 576 MB
> >>>>
> >>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>> X = 1MB (11M events total)
> >>>> Test1:
> >>>> Flush Time = 52 mS
> >>>> Scan Time = 19 mS
> >>>> 7.2 GB
> >>>>
> >>>> Test2:
> >>>> Flush Time = 84 mS
> >>>> Scan Time = 34 mS
> >>>> 9.1 GB
> >>>>
> >>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>> X = 2.5M (27.5M events total)
> >>>> Test1:
> >>>> Flush Time = 82 mS
> >>>> Scan Time = 63 mS
> >>>> 17GB - 276 sst files
> >>>>
> >>>> Test2:
> >>>> Flush Time = 116 mS
> >>>> Scan Time = 35 mS
> >>>> 23GB - 361 sst files
> >>>>
> >>>> Test3:
> >>>> Flush Time = 103 mS
> >>>> Scan Time = 82 mS
> >>>> 19 GB - 300 sst files
> >>>>
> >>>>
> >>>
> --------------------------------------------------------------------------------------------
> >>>>
> >>>> I had to limit my testing on my laptop to X = 2.5M events. I tried to
> go
> >>>> to X = 10M (110M events) but RocksDB was going into the 100GB+ range
> >>> and my
> >>>> laptop ran out of disk. More extensive testing could be done but I
> >>> suspect
> >>>> that it would be in line with what we're seeing in the results above.
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> At this point in time, I think the only major discussion point is
> really
> >>>> around what Jan and I have disagreed on: repartitioning back +
> resolving
> >>>> potential out of order issues or leaving that up to the client to
> >>> handle.
> >>>>
> >>>>
> >>>> Thanks folks,
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Jan.Filipiak@trivago.com
> >
> >>>> wrote:
> >>>>
> >>>>>
> >>>>>
> >>>>> On 29.11.2018 15:14, John Roesler wrote:
> >>>>>> Hi all,
> >>>>>>
> >>>>>> Sorry that this discussion petered out... I think the 2.1 release
> >>>>> caused an
> >>>>>> extended distraction that pushed it off everyone's radar (which was
> >>>>>> precisely Adam's concern). Personally, I've also had some extend
> >>>>>> distractions of my own that kept (and continue to keep) me
> >>> preoccupied.
> >>>>>>
> >>>>>> However, calling for a vote did wake me up, so I guess Jan was on
> the
> >>>>> right
> >>>>>> track!
> >>>>>>
> >>>>>> I've gone back and reviewed the whole KIP document and the prior
> >>>>>> discussion, and I'd like to offer a few thoughts:
> >>>>>>
> >>>>>> API Thoughts:
> >>>>>>
> >>>>>> 1. If I read the KIP right, you are proposing a many-to-one join.
> >>> Could
> >>>>> we
> >>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the design
> >>>>> around
> >>>>>> and make it a oneToManyJoin?
> >>>>>>
> >>>>>> The proposed name "joinOnForeignKey" disguises the join type, and it
> >>>>> seems
> >>>>>> like it might trick some people into using it for a one-to-one join.
> >>>>> This
> >>>>>> would work, of course, but it would be super inefficient compared to
> >>> a
> >>>>>> simple rekey-and-join.
> >>>>>>
> >>>>>> 2. I might have missed it, but I don't think it's specified whether
> >>>>> it's an
> >>>>>> inner, outer, or left join. I'm guessing an outer join, as
> >>> (neglecting
> >>>>> IQ),
> >>>>>> the rest can be achieved by filtering or by handling it in the
> >>>>> ValueJoiner.
> >>>>>>
> >>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
> >>>>>> 3a. Regarding Serialized: There are a few different paradigms in
> >>> play in
> >>>>>> the Streams API, so it's confusing, but instead of three Serialized
> >>>>> args, I
> >>>>>> think it would be better to have one that allows (optionally)
> setting
> >>>>> the 4
> >>>>>> incoming serdes. The result serde is defined by the Materialized.
> The
> >>>>>> incoming serdes can be optional because they might already be
> >>> available
> >>>>> on
> >>>>>> the source KTables, or the default serdes from the config might be
> >>>>>> applicable.
> >>>>>>
> >>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't allow
> >>>>> setting
> >>>>>> one, and it seems like it might actually be harmful, since the rekey
> >>>>>> operation needs to produce results that are co-partitioned with the
> >>>>> "other"
> >>>>>> KTable.
> >>>>>>
> >>>>>> 4. I'm fine with the "reserved word" header, but I didn't actually
> >>>>> follow
> >>>>>> what Matthias meant about namespacing requiring "deserializing" the
> >>>>> record
> >>>>>> header. The headers are already Strings, so I don't think that
> >>>>>> deserialization is required. If we applied the namespace at source
> >>> nodes
> >>>>>> and stripped it at sink nodes, this would be practically no
> overhead.
> >>>>> The
> >>>>>> advantage of the namespace idea is that no public API change wrt
> >>> headers
> >>>>>> needs to happen, and no restrictions need to be placed on users'
> >>>>> headers.
> >>>>>>
> >>>>>> (Although I'm wondering if we can get away without the header at
> >>> all...
> >>>>>> stay tuned)
> >>>>>>
> >>>>>> 5. I also didn't follow the discussion about the HWM table growing
> >>>>> without
> >>>>>> bound. As I read it, the HWM table is effectively implementing OCC
> to
> >>>>>> resolve the problem you noted with disordering when the rekey is
> >>>>>> reversed... particularly notable when the FK changes. As such, it
> >>> only
> >>>>>> needs to track the most recent "version" (the offset in the source
> >>>>>> partition) of each key. Therefore, it should have the same number of
> >>>>> keys
> >>>>>> as the source table at all times.
> >>>>>>
> >>>>>> I see that you are aware of KIP-258, which I think might be relevant
> >>> in
> >>>>> a
> >>>>>> couple of ways. One: it's just about storing the timestamp in the
> >>> state
> >>>>>> store, but the ultimate idea is to effectively use the timestamp as
> >>> an
> >>>>> OCC
> >>>>>> "version" to drop disordered updates. You wouldn't want to use the
> >>>>>> timestamp for this operation, but if you were to use a similar
> >>>>> mechanism to
> >>>>>> store the source offset in the store alongside the re-keyed values,
> >>> then
> >>>>>> you could avoid a separate table.
> >>>>>>
> >>>>>> 6. You and Jan have been thinking about this for a long time, so
> I've
> >>>>>> probably missed something here, but I'm wondering if we can avoid
> the
> >>>>> HWM
> >>>>>> tracking at all and resolve out-of-order during a final join
> >>> instead...
> >>>>>>
> >>>>>> Let's say we're joining a left table (Integer K: Letter FK, (other
> >>>>> data))
> >>>>>> to a right table (Letter K: (some data)).
> >>>>>>
> >>>>>> Left table:
> >>>>>> 1: (A, xyz)
> >>>>>> 2: (B, asd)
> >>>>>>
> >>>>>> Right table:
> >>>>>> A: EntityA
> >>>>>> B: EntityB
> >>>>>>
> >>>>>> We could do a rekey as you proposed with a combined key, but not
> >>>>>> propagating the value at all..
> >>>>>> Rekey table:
> >>>>>> A-1: (dummy value)
> >>>>>> B-2: (dummy value)
> >>>>>>
> >>>>>> Which we then join with the right table to produce:
> >>>>>> A-1: EntityA
> >>>>>> B-2: EntityB
> >>>>>>
> >>>>>> Which gets rekeyed back:
> >>>>>> 1: A, EntityA
> >>>>>> 2: B, EntityB
> >>>>>>
> >>>>>> And finally we do the actual join:
> >>>>>> Result table:
> >>>>>> 1: ((A, xyz), EntityA)
> >>>>>> 2: ((B, asd), EntityB)
> >>>>>>
> >>>>>> The thing is that in that last join, we have the opportunity to
> >>> compare
> >>>>> the
> >>>>>> current FK in the left table with the incoming PK of the right
> >>> table. If
> >>>>>> they don't match, we just drop the event, since it must be outdated.
> >>>>>>
> >>>>>
> >>>>>> In your KIP, you gave an example in which (1: A, xyz) gets updated
> to
> >>>>> (1:
> >>>>>> B, xyz), ultimately yielding a conundrum about whether the final
> >>> state
> >>>>>> should be (1: null) or (1: joined-on-B). With the algorithm above,
> >>> you
> >>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> >>>>>> EntityB)). It seems like this does give you enough information to
> >>> make
> >>>>> the
> >>>>>> right choice, regardless of disordering.
> >>>>>
> >>>>> Will check Adams patch, but this should work. As mentioned often I am
> >>>>> not convinced on partitioning back for the user automatically. I
> think
> >>>>> this is the real performance eater ;)
> >>>>>
> >>>>>>
> >>>>>>
> >>>>>> 7. Last thought... I'm a little concerned about the performance of
> >>> the
> >>>>>> range scans when records change in the right table. You've said that
> >>>>> you've
> >>>>>> been using the algorithm you presented in production for a while.
> Can
> >>>>> you
> >>>>>> give us a sense of the performance characteristics you've observed?
> >>>>>>
> >>>>>
> >>>>> Make it work, make it fast, make it beautiful. The topmost thing here
> >>> is
> >>>>> / was correctness. In practice I do not measure the performance of
> the
> >>>>> range scan. Usual cases I run this with is emitting 500k - 1kk rows
> >>>>> on a left hand side change. The range scan is just the work you gotta
> >>>>> do, also when you pack your data into different formats, usually the
> >>>>> rocks performance is very tight to the size of the data and we can't
> >>>>> really change that. It is more important for users to prevent useless
> >>>>> updates to begin with. My left hand side is guarded to drop changes
> >>> that
> >>>>> are not going to change my join output.
> >>>>>
> >>>>> usually it's:
> >>>>>
> >>>>> drop unused fields and then don't forward if old.equals(new)
> >>>>>
> >>>>> regarding to the performance of creating an iterator for smaller
> >>>>> fanouts, users can still just do a group by first then anyways.
> >>>>>
> >>>>>
> >>>>>
> >>>>>> I could only think of one alternative, but I'm not sure if it's
> >>> better
> >>>>> or
> >>>>>> worse... If the first re-key only needs to preserve the original
> key,
> >>>>> as I
> >>>>>> proposed in #6, then we could store a vector of keys in the value:
> >>>>>>
> >>>>>> Left table:
> >>>>>> 1: A,...
> >>>>>> 2: B,...
> >>>>>> 3: A,...
> >>>>>>
> >>>>>> Gets re-keyed:
> >>>>>> A: [1, 3]
> >>>>>> B: [2]
> >>>>>>
> >>>>>> Then, the rhs part of the join would only need a regular single-key
> >>>>> lookup.
> >>>>>> Of course we have to deal with the problem of large values, as
> >>> there's
> >>>>> no
> >>>>>> bound on the number of lhs records that can reference rhs records.
> >>>>> Offhand,
> >>>>>> I'd say we could page the values, so when one row is past the
> >>>>> threshold, we
> >>>>>> append the key for the next page. Then in most cases, it would be a
> >>>>> single
> >>>>>> key lookup, but for large fan-out updates, it would be one per (max
> >>>>> value
> >>>>>> size)/(avg lhs key size).
> >>>>>>
> >>>>>> This seems more complex, though... Plus, I think there's some extra
> >>>>>> tracking we'd need to do to know when to emit a retraction. For
> >>> example,
> >>>>>> when record 1 is deleted, the re-key table would just have (A: [3]).
> >>>>> Some
> >>>>>> kind of tombstone is needed so that the join result for 1 can also
> be
> >>>>>> retracted.
> >>>>>>
> >>>>>> That's all!
> >>>>>>
> >>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry
> the
> >>>>>> discussion has been slow.
> >>>>>> -John
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> >>> Jan.Filipiak@trivago.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Id say you can just call the vote.
> >>>>>>>
> >>>>>>> that happens all the time, and if something comes up, it just goes
> >>> back
> >>>>>>> to discuss.
> >>>>>>>
> >>>>>>> would not expect to much attention with another another email in
> >>> this
> >>>>>>> thread.
> >>>>>>>
> >>>>>>> best Jan
> >>>>>>>
> >>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
> >>>>>>>> Hello Contributors
> >>>>>>>>
> >>>>>>>> I know that 2.1 is about to be released, but I do need to bump
> >>> this to
> >>>>>>> keep
> >>>>>>>> visibility up. I am still intending to push this through once
> >>>>> contributor
> >>>>>>>> feedback is given.
> >>>>>>>>
> >>>>>>>> Main points that need addressing:
> >>>>>>>> 1) Any way (or benefit) in structuring the current singular graph
> >>> node
> >>>>>>> into
> >>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I am a
> >>> bit
> >>>>>>> fuzzy
> >>>>>>>> on how the optimizations are supposed to work, so I would
> >>> appreciate
> >>>>> any
> >>>>>>>> help on this aspect.
> >>>>>>>>
> >>>>>>>> 2) Overall strategy for joining + resolving. This thread has much
> >>>>>>> discourse
> >>>>>>>> between Jan and I between the current highwater mark proposal and
> a
> >>>>>>> groupBy
> >>>>>>>> + reduce proposal. I am of the opinion that we need to strictly
> >>> handle
> >>>>>>> any
> >>>>>>>> chance of out-of-order data and leave none of it up to the
> >>> consumer.
> >>>>> Any
> >>>>>>>> comments or suggestions here would also help.
> >>>>>>>>
> >>>>>>>> 3) Anything else that you see that would prevent this from moving
> >>> to a
> >>>>>>> vote?
> >>>>>>>>
> >>>>>>>> Thanks
> >>>>>>>>
> >>>>>>>> Adam
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >>>>>>> adam.bellemare@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Jan
> >>>>>>>>>
> >>>>>>>>> With the Stores.windowStoreBuilder and
> >>> Stores.persistentWindowStore,
> >>>>> you
> >>>>>>>>> actually only need to specify the amount of segments you want and
> >>> how
> >>>>>>> large
> >>>>>>>>> they are. To the best of my understanding, what happens is that
> >>> the
> >>>>>>>>> segments are automatically rolled over as new data with new
> >>>>> timestamps
> >>>>>>> are
> >>>>>>>>> created. We use this exact functionality in some of the work done
> >>>>>>>>> internally at my company. For reference, this is the hopping
> >>> windowed
> >>>>>>> store.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>>>>>>>>
> >>>>>>>>> In the code that I have provided, there are going to be two 24h
> >>>>>>> segments.
> >>>>>>>>> When a record is put into the windowStore, it will be inserted at
> >>>>> time
> >>>>>>> T in
> >>>>>>>>> both segments. The two segments will always overlap by 12h. As
> >>> time
> >>>>>>> goes on
> >>>>>>>>> and new records are added (say at time T+12h+), the oldest
> segment
> >>>>> will
> >>>>>>> be
> >>>>>>>>> automatically deleted and a new segment created. The records are
> >>> by
> >>>>>>> default
> >>>>>>>>> inserted with the context.timestamp(), such that it is the record
> >>>>> time,
> >>>>>>> not
> >>>>>>>>> the clock time, which is used.
> >>>>>>>>>
> >>>>>>>>> To the best of my understanding, the timestamps are retained when
> >>>>>>>>> restoring from the changelog.
> >>>>>>>>>
> >>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
> >>>>> segment-level,
> >>>>>>>>> instead of at an individual record level.
> >>>>>>>>>
> >>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >>>>> Jan.Filipiak@trivago.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Will that work? I expected it to blow up with ClassCastException
> >>> or
> >>>>>>>>>> similar.
> >>>>>>>>>>
> >>>>>>>>>> You either would have to specify the window you fetch/put or
> >>> iterate
> >>>>>>>>>> across all windows the key was found in right?
> >>>>>>>>>>
> >>>>>>>>>> I just hope the window-store doesn't check stream-time under the
> >>>>> hoods
> >>>>>>>>>> that would be a questionable interface.
> >>>>>>>>>>
> >>>>>>>>>> If it does: did you see my comment on checking all the windows
> >>>>> earlier?
> >>>>>>>>>> that would be needed to actually give reasonable time gurantees.
> >>>>>>>>>>
> >>>>>>>>>> Best
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>>>>>>>>> Hi Jan
> >>>>>>>>>>>
> >>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the state
> >>>>> store,
> >>>>>>>>>> not
> >>>>>>>>>>> the ProcessorSupplier.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Adam
> >>>>>>>>>>>
> >>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> @Guozhang
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for the information. This is indeed something that
> >>> will be
> >>>>>>>>>>>>> extremely
> >>>>>>>>>>>>> useful for this KIP.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>> Thanks for your explanations. That being said, I will not be
> >>>>> moving
> >>>>>>>>>> ahead
> >>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution as
> you
> >>>>>>>>>> propose.
> >>>>>>>>>>>>> That being said, if you wish to implement it yourself off of
> >>> my
> >>>>>>>>>> current PR
> >>>>>>>>>>>>> and submit it as a competitive alternative, I would be more
> >>> than
> >>>>>>>>>> happy to
> >>>>>>>>>>>>> help vet that as an alternate solution. As it stands right
> >>> now,
> >>>>> I do
> >>>>>>>>>> not
> >>>>>>>>>>>>> really have more time to invest into alternatives without
> >>> there
> >>>>>>> being
> >>>>>>>>>> a
> >>>>>>>>>>>>> strong indication from the binding voters which they would
> >>>>> prefer.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>> Hey, total no worries. I think I personally gave up on the
> >>> streams
> >>>>>>> DSL
> >>>>>>>>>> for
> >>>>>>>>>>>> some time already, otherwise I would have pulled this KIP
> >>> through
> >>>>>>>>>> already.
> >>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>> I will look at finishing up my PR with the windowed state
> >>> store
> >>>>> in
> >>>>>>> the
> >>>>>>>>>>>>> next
> >>>>>>>>>>>>> week or so, exercising it via tests, and then I will come
> back
> >>>>> for
> >>>>>>>>>> final
> >>>>>>>>>>>>> discussions. In the meantime, I hope that any of the binding
> >>>>> voters
> >>>>>>>>>> could
> >>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
> >>> according
> >>>>> to
> >>>>>>> the
> >>>>>>>>>>>>> latest plan:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>>>>>>>>> Support+non-key+joining+in+KTable
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store. This
> >>>>> could
> >>>>>>> be
> >>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
> >>> completed.
> >>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated
> >>> in
> >>>>> the
> >>>>>>>>>> PR?
> >>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing something?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> >>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong
> link,
> >>>>> as it
> >>>>>>>>>> is
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258
> >>> we do
> >>>>>>>>>> want to
> >>>>>>>>>>>>>> have "handling out-of-order data for source KTable" such
> that
> >>>>>>>>>> instead of
> >>>>>>>>>>>>>> blindly apply the updates to the materialized store, i.e.
> >>>>> following
> >>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>> ordering, we will reject updates that are older than the
> >>> current
> >>>>>>>>>> key's
> >>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> >>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hello Adam,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
> >>> the
> >>>>>>> high
> >>>>>>>>>>>>>>> watermark store, now altered to be replaced with a window
> >>>>> store),
> >>>>>>> I
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>> another current on-going KIP may actually help:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> This is for adding the timestamp into a key-value store
> >>> (i.e.
> >>>>> only
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
> >>> described
> >>>>> in
> >>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that
> >>> we
> >>>>> can
> >>>>>>>>>> then
> >>>>>>>>>>>>>>> "reject" updates from the source topics if its timestamp is
> >>>>>>> smaller
> >>>>>>>>>> than
> >>>>>>>>>>>>>>> the current key's latest update timestamp. I think it is
> >>> very
> >>>>>>>>>> similar to
> >>>>>>>>>>>>>>> what you have in mind for high watermark based filtering,
> >>> while
> >>>>>>> you
> >>>>>>>>>> only
> >>>>>>>>>>>>>>> need to make sure that the timestamps of the joining
> records
> >>>>> are
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> correctly
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> inherited though the whole topology to the final stage.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
> >>>>> non-windowed
> >>>>>>>>>> KTables
> >>>>>>>>>>>>>>> only, but for windowed KTables we do not really have a good
> >>>>>>> support
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> their joins anyways (
> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
> >>> non-key
> >>>>>>> joins
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >>>>>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Current highwater mark implementation would grow
> endlessly
> >>>>> based
> >>>>>>>>>> on
> >>>>>>>>>>>>>>>>> primary key of original event. It is a pair of (<this
> >>> table
> >>>>>>>>>> primary
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> key>,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
> >>>>> differentiate
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal would be
> >>> to
> >>>>>>>>>> replace
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would allow
> >>> the
> >>>>>>> same
> >>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This should
> >>> allow
> >>>>> for
> >>>>>>>>>> all
> >>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
> >>>>> customizable
> >>>>>>>>>> by
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
> >>>>> minutes
> >>>>>>> of
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> window,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the trick
> here.
> >>>>> Even
> >>>>>>>>>> if I
> >>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
> >>> optional
> >>>>>>>>>> since I
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a little bit
> >>>>>>>>>> sceptical
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> how to determine the window. So esentially one could run
> >>> into
> >>>>>>>>>> problems
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> the rapid change happens near a window border. I will check
> >>> you
> >>>>>>>>>>>>>>>> implementation in detail, if its problematic, we could
> >>> still
> >>>>>>> check
> >>>>>>>>>>>>>>>> _all_
> >>>>>>>>>>>>>>>> windows on read with not to bad performance impact I
> guess.
> >>>>> Will
> >>>>>>>>>> let
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
> >>> wouldn't
> >>>>> not
> >>>>>>>>>> like
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> >>>>>>> timestamp(B).
> >>>>>>>>>> I
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> we can't expect that.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> @Jan
> >>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks for the
> >>>>>>>>>> diagram, it
> >>>>>>>>>>>>>>>>> did really help. You are correct that I do not have the
> >>>>> original
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> primary
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> key available, and I can see that if it was available then
> >>> you
> >>>>>>>>>> would be
> >>>>>>>>>>>>>>>>> able to add and remove events from the Map. That being
> >>> said,
> >>>>> I
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> encourage
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity for
> >>>>> everyone
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> else.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But
> >>> I
> >>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original
> primary
> >>>>> key,
> >>>>>>> We
> >>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and
> basically
> >>>>> not
> >>>>>>>>>> using
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
> >>> original
> >>>>> DSL
> >>>>>>>>>> its
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my end.
> >>> Will
> >>>>>>>>>> finish
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay
> inside
> >>>>> the
> >>>>>>>>>> State
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
> >>> propagated?
> >>>>>>> Isn't
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> effectively the same as a highwater mark state store?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
> >>>>> return
> >>>>>>>>>> `null`
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> the key is removed from the keyspace. But there is going to
> >>> be
> >>>>> a
> >>>>>>>>>> store
> >>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store directly
> >>> for
> >>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
> >>> store,
> >>>>>>>>>> satisfying
> >>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
> >>> Windowed
> >>>>>>>>>> store is
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> keeping the values, so for the next statefull operation we
> >>>>> would
> >>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the window
> >>>>> store
> >>>>>>>>>> also
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> the values then.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group by
> >>> before
> >>>>>>>>>>>>>>>> repartitioning to the original primary key i think it
> would
> >>>>> help
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> big time in building efficient apps. Given the original
> >>> primary
> >>>>>>> key
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> issue I
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> understand that we do not have a solid foundation to build
> >>> on.
> >>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
> >>>>> unfortunate. I
> >>>>>>>>>> could
> >>>>>>>>>>>>>>>> understand the decision goes like that. I do not think its
> >>> a
> >>>>> good
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> decision.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>> Adam
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >>>>> dumbreprajakta311@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          please remove me from this group
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
> >>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > Hi Adam,
> >>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>          > give me some time, will make such a chart.
> last
> >>>>> time i
> >>>>>>>>>> didn't
> >>>>>>>>>>>>>>>>>          get along
> >>>>>>>>>>>>>>>>>          > well with giphy and ruined all your charts.
> >>>>>>>>>>>>>>>>>          > Hopefully i can get it done today
> >>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>          > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>>>>>>>>>>>>>>>          > > Hi Jan
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > > I have included a diagram of what I
> attempted
> >>> on
> >>>>> the
> >>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>>>>>>>>>
> >>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>>>>>>>>          <
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>>>>>>>>>
> >>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > > I attempted this back at the start of my own
> >>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>          this
> >>>>>>>>>>>>>>>>>          > > solution, and since I could not get it to
> >>> work I
> >>>>> have
> >>>>>>>>>> since
> >>>>>>>>>>>>>>>>>          discarded the
> >>>>>>>>>>>>>>>>>          > > code. At this point in time, if you wish to
> >>>>> continue
> >>>>>>>>>> pursuing
> >>>>>>>>>>>>>>>>>          for your
> >>>>>>>>>>>>>>>>>          > > groupBy solution, I ask that you please
> >>> create a
> >>>>>>>>>> diagram on
> >>>>>>>>>>>>>>>>>          the KIP
> >>>>>>>>>>>>>>>>>          > > carefully explaining your solution. Please
> >>> feel
> >>>>> free
> >>>>>>> to
> >>>>>>>>>> use
> >>>>>>>>>>>>>>>>>          the image I
> >>>>>>>>>>>>>>>>>          > > just posted as a starting point. I am having
> >>>>> trouble
> >>>>>>>>>>>>>>>>>          understanding your
> >>>>>>>>>>>>>>>>>          > > explanations but I think that a carefully
> >>>>> constructed
> >>>>>>>>>> diagram
> >>>>>>>>>>>>>>>>>          will clear
> >>>>>>>>>>>>>>>>>          > up
> >>>>>>>>>>>>>>>>>          > > any misunderstandings. Alternately, please
> >>> post a
> >>>>>>>>>>>>>>>>>          comprehensive PR with
> >>>>>>>>>>>>>>>>>          > > your solution. I can only guess at what you
> >>>>> mean, and
> >>>>>>>>>> since I
> >>>>>>>>>>>>>>>>>          value my
> >>>>>>>>>>>>>>>>>          > own
> >>>>>>>>>>>>>>>>>          > > time as much as you value yours, I believe
> it
> >>> is
> >>>>> your
> >>>>>>>>>>>>>>>>>          responsibility to
> >>>>>>>>>>>>>>>>>          > > provide an implementation instead of me
> >>> trying to
> >>>>>>> guess.
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > > Adam
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
> >>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > > wrote:
> >>>>>>>>>>>>>>>>>          > >
> >>>>>>>>>>>>>>>>>          > >> Hi James,
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >> nice to see you beeing interested. kafka
> >>>>> streams at
> >>>>>>>>>> this
> >>>>>>>>>>>>>>>>>          point supports
> >>>>>>>>>>>>>>>>>          > >> all sorts of joins as long as both streams
> >>> have
> >>>>> the
> >>>>>>>>>> same
> >>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>          > >> Adam is currently implementing a join
> where a
> >>>>> KTable
> >>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>          KTable can
> >>>>>>>>>>>>>>>>>          > have
> >>>>>>>>>>>>>>>>>          > >> a one to many relation ship (1:n). We
> exploit
> >>>>> that
> >>>>>>>>>> rocksdb
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > >> datastore that keeps data sorted (At least
> >>>>> exposes an
> >>>>>>>>>> API to
> >>>>>>>>>>>>>>>>>          access the
> >>>>>>>>>>>>>>>>>          > >> stored data in a sorted fashion).
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >> I think the technical caveats are well
> >>>>> understood
> >>>>>>> now
> >>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > basically
> >>>>>>>>>>>>>>>>>          > >> down to philosophy and API Design ( when
> Adam
> >>>>> sees
> >>>>>>> my
> >>>>>>>>>> newest
> >>>>>>>>>>>>>>>>>          message).
> >>>>>>>>>>>>>>>>>          > >> I have a lengthy track record of loosing
> >>> those
> >>>>> kinda
> >>>>>>>>>>>>>>>>>          arguments within
> >>>>>>>>>>>>>>>>>          > the
> >>>>>>>>>>>>>>>>>          > >> streams community and I have no clue why.
> So
> >>> I
> >>>>>>>>>> literally
> >>>>>>>>>>>>>>>>>          can't wait for
> >>>>>>>>>>>>>>>>>          > you
> >>>>>>>>>>>>>>>>>          > >> to churn through this thread and give you
> >>>>> opinion on
> >>>>>>>>>> how we
> >>>>>>>>>>>>>>>>>          should
> >>>>>>>>>>>>>>>>>          > design
> >>>>>>>>>>>>>>>>>          > >> the return type of the oneToManyJoin and
> how
> >>>>> many
> >>>>>>>>>> power we
> >>>>>>>>>>>>>>>>>          want to give
> >>>>>>>>>>>>>>>>>          > to
> >>>>>>>>>>>>>>>>>          > >> the user vs "simplicity" (where simplicity
> >>> isn't
> >>>>>>>>>> really that
> >>>>>>>>>>>>>>>>>          as users
> >>>>>>>>>>>>>>>>>          > still
> >>>>>>>>>>>>>>>>>          > >> need to understand it I argue)
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >> waiting for you to join in on the
> discussion
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >> Best Jan
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>>>>>>>>>>>>>>>          > >>
> >>>>>>>>>>>>>>>>>          > >>> I am new to this group and I found this
> >>> subject
> >>>>>>>>>>>>>>>>>          interesting.  Sounds
> >>>>>>>>>>>>>>>>>          > like
> >>>>>>>>>>>>>>>>>          > >>> you guys want to implement a join table of
> >>> two
> >>>>>>>>>> streams? Is
> >>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>          > somewhere
> >>>>>>>>>>>>>>>>>          > >>> I can see the original requirement or
> >>> proposal?
> >>>>>>>>>>>>>>>>>          > >>>
> >>>>>>>>>>>>>>>>>          > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
> >>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>> On 05.09.2018 22:17, Adam Bellemare
> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>> I'm currently testing using a Windowed
> >>> Store
> >>>>> to
> >>>>>>>>>> store the
> >>>>>>>>>>>>>>>>>          highwater
> >>>>>>>>>>>>>>>>>          > >>>>> mark.
> >>>>>>>>>>>>>>>>>          > >>>>> By all indications this should work
> fine,
> >>>>> with
> >>>>>>> the
> >>>>>>>>>> caveat
> >>>>>>>>>>>>>>>>>          being that
> >>>>>>>>>>>>>>>>>          > it
> >>>>>>>>>>>>>>>>>          > >>>>> can
> >>>>>>>>>>>>>>>>>          > >>>>> only resolve out-of-order arrival for up
> >>> to
> >>>>> the
> >>>>>>>>>> size of
> >>>>>>>>>>>>>>>>>          the window
> >>>>>>>>>>>>>>>>>          > (ie:
> >>>>>>>>>>>>>>>>>          > >>>>> 24h, 72h, etc). This would remove the
> >>>>> possibility
> >>>>>>>>>> of it
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > unbounded
> >>>>>>>>>>>>>>>>>          > >>>>> in
> >>>>>>>>>>>>>>>>>          > >>>>> size.
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>> With regards to Jan's suggestion, I
> >>> believe
> >>>>> this
> >>>>>>> is
> >>>>>>>>>> where
> >>>>>>>>>>>>>>>>>          we will
> >>>>>>>>>>>>>>>>>          > have
> >>>>>>>>>>>>>>>>>          > >>>>> to
> >>>>>>>>>>>>>>>>>          > >>>>> remain in disagreement. While I do not
> >>>>> disagree
> >>>>>>>>>> with your
> >>>>>>>>>>>>>>>>>          statement
> >>>>>>>>>>>>>>>>>          > >>>>> about
> >>>>>>>>>>>>>>>>>          > >>>>> there likely to be additional joins done
> >>> in a
> >>>>>>>>>> real-world
> >>>>>>>>>>>>>>>>>          workflow, I
> >>>>>>>>>>>>>>>>>          > do
> >>>>>>>>>>>>>>>>>          > >>>>> not
> >>>>>>>>>>>>>>>>>          > >>>>> see how you can conclusively deal with
> >>>>>>> out-of-order
> >>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>          > >>>>> foreign-key
> >>>>>>>>>>>>>>>>>          > >>>>> changes and subsequent joins. I have
> >>>>> attempted
> >>>>>>> what
> >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>          think you have
> >>>>>>>>>>>>>>>>>          > >>>>> proposed (without a high-water, using
> >>>>> groupBy and
> >>>>>>>>>> reduce)
> >>>>>>>>>>>>>>>>>          and found
> >>>>>>>>>>>>>>>>>          > >>>>> that if
> >>>>>>>>>>>>>>>>>          > >>>>> the foreign key changes too quickly, or
> >>> the
> >>>>> load
> >>>>>>> on
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>>          stream thread
> >>>>>>>>>>>>>>>>>          > is
> >>>>>>>>>>>>>>>>>          > >>>>> too
> >>>>>>>>>>>>>>>>>          > >>>>> high, the joined messages will arrive
> >>>>>>> out-of-order
> >>>>>>>>>> and be
> >>>>>>>>>>>>>>>>>          incorrectly
> >>>>>>>>>>>>>>>>>          > >>>>> propagated, such that an intermediate
> >>> event
> >>>>> is
> >>>>>>>>>>>>>>>>> represented
> >>>>>>>>>>>>>>>>>          as the
> >>>>>>>>>>>>>>>>>          > final
> >>>>>>>>>>>>>>>>>          > >>>>> event.
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>> Can you shed some light on your groupBy
> >>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>>          There must be
> >>>>>>>>>>>>>>>>>          > >>>> some sort of flaw in it.
> >>>>>>>>>>>>>>>>>          > >>>> I have a suspicion where it is, I would
> >>> just
> >>>>> like
> >>>>>>> to
> >>>>>>>>>>>>>>>>>          confirm. The idea
> >>>>>>>>>>>>>>>>>          > >>>> is bullet proof and it must be
> >>>>>>>>>>>>>>>>>          > >>>> an implementation mess up. I would like
> to
> >>>>> clarify
> >>>>>>>>>> before
> >>>>>>>>>>>>>>>>>          we draw a
> >>>>>>>>>>>>>>>>>          > >>>> conclusion.
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>    Repartitioning the scattered events
> >>> back to
> >>>>>>> their
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > >>>>> partitions is the only way I know how to
> >>>>>>> conclusively
> >>>>>>>>>> deal
> >>>>>>>>>>>>>>>>>          with
> >>>>>>>>>>>>>>>>>          > >>>>> out-of-order events in a given time
> frame,
> >>>>> and to
> >>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>          that the
> >>>>>>>>>>>>>>>>>          > data
> >>>>>>>>>>>>>>>>>          > >>>>> is
> >>>>>>>>>>>>>>>>>          > >>>>> eventually consistent with the input
> >>> events.
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>> If you have some code to share that
> >>>>> illustrates
> >>>>>>> your
> >>>>>>>>>>>>>>>>>          approach, I
> >>>>>>>>>>>>>>>>>          > would
> >>>>>>>>>>>>>>>>>          > >>>>> be
> >>>>>>>>>>>>>>>>>          > >>>>> very grateful as it would remove any
> >>>>>>>>>> misunderstandings
> >>>>>>>>>>>>>>>>>          that I may
> >>>>>>>>>>>>>>>>>          > have.
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>> ah okay you were looking for my code. I
> >>> don't
> >>>>> have
> >>>>>>>>>>>>>>>>>          something easily
> >>>>>>>>>>>>>>>>>          > >>>> readable here as its bloated with
> >>> OO-patterns.
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>> its anyhow trivial:
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>> @Override
> >>>>>>>>>>>>>>>>>          > >>>>      public T apply(K aggKey, V value, T
> >>>>>>> aggregate)
> >>>>>>>>>>>>>>>>>          > >>>>      {
> >>>>>>>>>>>>>>>>>          > >>>>          Map<U, V> currentStateAsMap =
> >>>>>>>>>> asMap(aggregate);
> >>>>>>>>>>>>>>>>> <<
> >>>>>>>>>>>>>>>>>          imaginary
> >>>>>>>>>>>>>>>>>          > >>>>          U toModifyKey =
> >>> mapper.apply(value);
> >>>>>>>>>>>>>>>>>          > >>>>              << this is the place where
> >>> people
> >>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>          gonna have
> >>>>>>>>>>>>>>>>>          > issues
> >>>>>>>>>>>>>>>>>          > >>>> and why you probably couldn't do it. we
> >>> would
> >>>>> need
> >>>>>>>>>> to find
> >>>>>>>>>>>>>>>>>          a solution
> >>>>>>>>>>>>>>>>>          > here.
> >>>>>>>>>>>>>>>>>          > >>>> I didn't realize that yet.
> >>>>>>>>>>>>>>>>>          > >>>>              << we propagate the field in
> >>> the
> >>>>>>>>>> joiner, so
> >>>>>>>>>>>>>>>>>          that we can
> >>>>>>>>>>>>>>>>>          > pick
> >>>>>>>>>>>>>>>>>          > >>>> it up in an aggregate. Probably you have
> >>> not
> >>>>>>> thought
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>>          this in your
> >>>>>>>>>>>>>>>>>          > >>>> approach right?
> >>>>>>>>>>>>>>>>>          > >>>>              << I am very open to find a
> >>>>> generic
> >>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>          here. In my
> >>>>>>>>>>>>>>>>>          > >>>> honest opinion this is broken in
> >>>>>>> KTableImpl.GroupBy
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>          looses
> >>>>>>>>>>>>>>>>>          > the keys
> >>>>>>>>>>>>>>>>>          > >>>> and only maintains the aggregate key.
> >>>>>>>>>>>>>>>>>          > >>>>              << I abstracted it away back
> >>>>> then way
> >>>>>>>>>> before
> >>>>>>>>>>>>>>>>> i
> >>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>          > thinking
> >>>>>>>>>>>>>>>>>          > >>>> of oneToMany join. That is why I didn't
> >>>>> realize
> >>>>>>> its
> >>>>>>>>>>>>>>>>>          significance here.
> >>>>>>>>>>>>>>>>>          > >>>>              << Opinions?
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>          for (V m : current)
> >>>>>>>>>>>>>>>>>          > >>>>          {
> >>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.put(mapper.apply(m),
> m);
> >>>>>>>>>>>>>>>>>          > >>>>          }
> >>>>>>>>>>>>>>>>>          > >>>>          if (isAdder)
> >>>>>>>>>>>>>>>>>          > >>>>          {
> >>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.put(toModifyKey,
> value);
> >>>>>>>>>>>>>>>>>          > >>>>          }
> >>>>>>>>>>>>>>>>>          > >>>>          else
> >>>>>>>>>>>>>>>>>          > >>>>          {
> >>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>>>>>>>>          > >>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>>>>>>>>          > >>>>                  return null;
> >>>>>>>>>>>>>>>>>          > >>>>              }
> >>>>>>>>>>>>>>>>>          > >>>>          }
> >>>>>>>>>>>>>>>>>          > >>>>          retrun
> >>>>> asAggregateType(currentStateAsMap)
> >>>>>>>>>>>>>>>>>          > >>>>      }
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>>
> >>>>>>>>>>>>>>>>>          > >>>> Thanks,
> >>>>>>>>>>>>>>>>>          > >>>>> Adam
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
> >>> Filipiak
> >>>>> <
> >>>>>>>>>>>>>>>>>          > Jan.Filipiak@trivago.com <mailto:
> >>>>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>>
> >>>>>>>>>>>>>>>>>          > >>>>> Thanks Adam for bringing Matthias to
> >>> speed!
> >>>>>>>>>>>>>>>>>          > >>>>>> about the differences. I think
> re-keying
> >>>>> back
> >>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>          optional at
> >>>>>>>>>>>>>>>>>          > >>>>>> best.
> >>>>>>>>>>>>>>>>>          > >>>>>> I would say we return a KScatteredTable
> >>> with
> >>>>>>>>>> reshuffle()
> >>>>>>>>>>>>>>>>>          returning
> >>>>>>>>>>>>>>>>>          > >>>>>> KTable<originalKey,Joined> to make the
> >>>>> backwards
> >>>>>>>>>>>>>>>>>          repartitioning
> >>>>>>>>>>>>>>>>>          > >>>>>> optional.
> >>>>>>>>>>>>>>>>>          > >>>>>> I am also in a big favour of doing the
> >>> out
> >>>>> of
> >>>>>>> order
> >>>>>>>>>>>>>>>>>          processing using
> >>>>>>>>>>>>>>>>>          > >>>>>> group
> >>>>>>>>>>>>>>>>>          > >>>>>> by instead high water mark tracking.
> >>>>>>>>>>>>>>>>>          > >>>>>> Just because unbounded growth is just
> >>> scary
> >>>>> + It
> >>>>>>>>>> saves
> >>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>          the header
> >>>>>>>>>>>>>>>>>          > >>>>>> stuff.
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>> I think the abstraction of always
> >>>>> repartitioning
> >>>>>>>>>> back is
> >>>>>>>>>>>>>>>>>          just not so
> >>>>>>>>>>>>>>>>>          > >>>>>> strong. Like the work has been done
> >>> before
> >>>>> we
> >>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>          back and
> >>>>>>>>>>>>>>>>>          > >>>>>> grouping
> >>>>>>>>>>>>>>>>>          > >>>>>> by something else afterwards is really
> >>>>> common.
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>> On 05.09.2018 13:49, Adam Bellemare
> >>> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>> Hi Matthias
> >>>>>>>>>>>>>>>>>          > >>>>>>> Thank you for your feedback, I do
> >>>>> appreciate
> >>>>>>> it!
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> While name spacing would be possible,
> it
> >>>>> would
> >>>>>>>>>> require
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>          > deserialize
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> user headers what implies a runtime
> >>>>> overhead.
> >>>>>>> I
> >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>          suggest to
> >>>>>>>>>>>>>>>>>          > no
> >>>>>>>>>>>>>>>>>          > >>>>>>>> namespace for now to avoid the
> >>> overhead.
> >>>>> If
> >>>>>>> this
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > problem in
> >>>>>>>>>>>>>>>>>          > >>>>>>>> the future, we can still add name
> >>> spacing
> >>>>>>> later
> >>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Agreed. I will go with using a
> reserved
> >>>>> string
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>          document it.
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> My main concern about the design it
> the
> >>>>> type of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          result KTable:
> >>>>>>>>>>>>>>>>>          > If
> >>>>>>>>>>>>>>>>>          > >>>>>>> I
> >>>>>>>>>>>>>>>>>          > >>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> In your example, you have table1 and
> >>> table2
> >>>>>>>>>> swapped.
> >>>>>>>>>>>>>>>>>          Here is how it
> >>>>>>>>>>>>>>>>>          > >>>>>>> works
> >>>>>>>>>>>>>>>>>          > >>>>>>> currently:
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> 1) table1 has the records that contain
> >>> the
> >>>>>>>>>> foreign key
> >>>>>>>>>>>>>>>>>          within their
> >>>>>>>>>>>>>>>>>          > >>>>>>> value.
> >>>>>>>>>>>>>>>>>          > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>          > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>          > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> 2) A Value mapper is required to
> extract
> >>>>> the
> >>>>>>>>>> foreign
> >>>>>>>>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>>          > >>>>>>> table1 foreign key mapper: ( value =>
> >>>>> value.fk
> >>>>>>>>>>>>>>>>>          <http://value.fk> )
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> The mapper is applied to each element
> in
> >>>>>>> table1,
> >>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>          new combined
> >>>>>>>>>>>>>>>>>          > >>>>>>> key is
> >>>>>>>>>>>>>>>>>          > >>>>>>> made:
> >>>>>>>>>>>>>>>>>          > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>,
> >>> <A-b,
> >>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>>>>>>>>          <B-c,
> >>>>>>>>>>>>>>>>>          > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> 3) The rekeyed events are
> copartitioned
> >>>>> with
> >>>>>>>>>> table2:
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>>>>>>>>>>>>          > >>>>>>> RepartitionedTable1: <A-a,
> >>> (fk=A,bar=1)>,
> >>>>> <A-b,
> >>>>>>>>>>>>>>>>>          (fk=A,bar=2)>
> >>>>>>>>>>>>>>>>>          > >>>>>>> Table2: <A,X>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>>>>>>>>>>>>          > >>>>>>> RepartitionedTable1: <B-c,
> (fk=B,bar=3)>
> >>>>>>>>>>>>>>>>>          > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> 4) From here, they can be joined
> >>> together
> >>>>>>> locally
> >>>>>>>>>> by
> >>>>>>>>>>>>>>>>>          applying the
> >>>>>>>>>>>>>>>>>          > >>>>>>> joiner
> >>>>>>>>>>>>>>>>>          > >>>>>>> function.
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> At this point, Jan's design and my
> >>> design
> >>>>>>>>>> deviate. My
> >>>>>>>>>>>>>>>>>          design goes
> >>>>>>>>>>>>>>>>>          > on
> >>>>>>>>>>>>>>>>>          > >>>>>>> to
> >>>>>>>>>>>>>>>>>          > >>>>>>> repartition the data post-join and
> >>> resolve
> >>>>>>>>>> out-of-order
> >>>>>>>>>>>>>>>>>          arrival of
> >>>>>>>>>>>>>>>>>          > >>>>>>> records,
> >>>>>>>>>>>>>>>>>          > >>>>>>> finally returning the data keyed just
> >>> the
> >>>>>>>>>> original key.
> >>>>>>>>>>>>>>>>>          I do not
> >>>>>>>>>>>>>>>>>          > >>>>>>> expose
> >>>>>>>>>>>>>>>>>          > >>>>>>> the
> >>>>>>>>>>>>>>>>>          > >>>>>>> CombinedKey or any of the internals
> >>>>> outside of
> >>>>>>> the
> >>>>>>>>>>>>>>>>>          joinOnForeignKey
> >>>>>>>>>>>>>>>>>          > >>>>>>> function. This does make for larger
> >>>>> footprint,
> >>>>>>>>>> but it
> >>>>>>>>>>>>>>>>>          removes all
> >>>>>>>>>>>>>>>>>          > >>>>>>> agency
> >>>>>>>>>>>>>>>>>          > >>>>>>> for resolving out-of-order arrivals
> and
> >>>>>>> handling
> >>>>>>>>>>>>>>>>>          CombinedKeys from
> >>>>>>>>>>>>>>>>>          > the
> >>>>>>>>>>>>>>>>>          > >>>>>>> user. I believe that this makes the
> >>>>> function
> >>>>>>> much
> >>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>          to use.
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> Let me know if this helps resolve your
> >>>>>>> questions,
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>          please feel
> >>>>>>>>>>>>>>>>>          > >>>>>>> free to
> >>>>>>>>>>>>>>>>>          > >>>>>>> add anything else on your mind.
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> Thanks again,
> >>>>>>>>>>>>>>>>>          > >>>>>>> Adam
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
> >>> Matthias J.
> >>>>>>> Sax <
> >>>>>>>>>>>>>>>>>          > >>>>>>> matthias@confluent.io <mailto:
> >>>>>>>>>> matthias@confluent.io>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>> Hi,
> >>>>>>>>>>>>>>>>>          > >>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> I am just catching up on this
> thread. I
> >>>>> did
> >>>>>>> not
> >>>>>>>>>> read
> >>>>>>>>>>>>>>>>>          everything so
> >>>>>>>>>>>>>>>>>          > >>>>>>>> far,
> >>>>>>>>>>>>>>>>>          > >>>>>>>> but want to share couple of initial
> >>>>> thoughts:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Headers: I think there is a
> fundamental
> >>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>          between header
> >>>>>>>>>>>>>>>>>          > >>>>>>>> usage
> >>>>>>>>>>>>>>>>>          > >>>>>>>> in this KIP and KP-258. For 258, we
> add
> >>>>>>> headers
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>          changelog topic
> >>>>>>>>>>>>>>>>>          > >>>>>>>> that
> >>>>>>>>>>>>>>>>>          > >>>>>>>> are owned by Kafka Streams and nobody
> >>>>> else is
> >>>>>>>>>> supposed
> >>>>>>>>>>>>>>>>>          to write
> >>>>>>>>>>>>>>>>>          > into
> >>>>>>>>>>>>>>>>>          > >>>>>>>> them. In fact, no user header are
> >>> written
> >>>>> into
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          changelog topic
> >>>>>>>>>>>>>>>>>          > >>>>>>>> and
> >>>>>>>>>>>>>>>>>          > >>>>>>>> thus, there are not conflicts.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Nevertheless, I don't see a big issue
> >>> with
> >>>>>>> using
> >>>>>>>>>>>>>>>>>          headers within
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Streams.
> >>>>>>>>>>>>>>>>>          > >>>>>>>> As long as we document it, we can
> have
> >>>>> some
> >>>>>>>>>> "reserved"
> >>>>>>>>>>>>>>>>>          header keys
> >>>>>>>>>>>>>>>>>          > >>>>>>>> and
> >>>>>>>>>>>>>>>>>          > >>>>>>>> users are not allowed to use when
> >>>>> processing
> >>>>>>>>>> data with
> >>>>>>>>>>>>>>>>>          Kafka
> >>>>>>>>>>>>>>>>>          > Streams.
> >>>>>>>>>>>>>>>>>          > >>>>>>>> IMHO, this should be ok.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> I think there is a safe way to avoid
> >>>>>>> conflicts,
> >>>>>>>>>> since
> >>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>          > headers
> >>>>>>>>>>>>>>>>>          > >>>>>>>> are
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> only needed in internal topics (I
> >>> think):
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> For internal and changelog topics,
> we
> >>> can
> >>>>>>>>>> namespace
> >>>>>>>>>>>>>>>>>          all headers:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> * user-defined headers are
> namespaced
> >>> as
> >>>>>>>>>> "external."
> >>>>>>>>>>>>>>>>> +
> >>>>>>>>>>>>>>>>>          headerKey
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> * internal headers are namespaced as
> >>>>>>>>>> "internal." +
> >>>>>>>>>>>>>>>>>          headerKey
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> While name spacing would be
> possible,
> >>> it
> >>>>>>> would
> >>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > >>>>>>>> deserialize
> >>>>>>>>>>>>>>>>>          > >>>>>>>> user headers what implies a runtime
> >>>>> overhead.
> >>>>>>> I
> >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>          suggest to
> >>>>>>>>>>>>>>>>>          > no
> >>>>>>>>>>>>>>>>>          > >>>>>>>> namespace for now to avoid the
> >>> overhead.
> >>>>> If
> >>>>>>> this
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> becomes a
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > problem in
> >>>>>>>>>>>>>>>>>          > >>>>>>>> the future, we can still add name
> >>> spacing
> >>>>>>> later
> >>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> My main concern about the design it
> the
> >>>>> type
> >>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          result KTable:
> >>>>>>>>>>>>>>>>>          > >>>>>>>> If I
> >>>>>>>>>>>>>>>>>          > >>>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K1,V3> joinedTable =
> >>>>>>>>>> table1.join(table2,...);
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> implies that the `joinedTable` has
> the
> >>>>> same
> >>>>>>> key
> >>>>>>>>>> as the
> >>>>>>>>>>>>>>>>>          left input
> >>>>>>>>>>>>>>>>>          > >>>>>>>> table.
> >>>>>>>>>>>>>>>>>          > >>>>>>>> IMHO, this does not work because if
> >>> table2
> >>>>>>>>>> contains
> >>>>>>>>>>>>>>>>>          multiple rows
> >>>>>>>>>>>>>>>>>          > >>>>>>>> that
> >>>>>>>>>>>>>>>>>          > >>>>>>>> join with a record in table1 (what is
> >>> the
> >>>>> main
> >>>>>>>>>> purpose
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>          > foreign
> >>>>>>>>>>>>>>>>>          > >>>>>>>> key
> >>>>>>>>>>>>>>>>>          > >>>>>>>> join), the result table would only
> >>>>> contain a
> >>>>>>>>>> single
> >>>>>>>>>>>>>>>>>          join result,
> >>>>>>>>>>>>>>>>>          > but
> >>>>>>>>>>>>>>>>>          > >>>>>>>> not
> >>>>>>>>>>>>>>>>>          > >>>>>>>> multiple.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Example:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> table1 input stream: <A,X>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> table2 input stream: <a,(A,1)>,
> >>> <b,(A,2)>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> We use table2 value a foreign key to
> >>>>> table1
> >>>>>>> key
> >>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>          "A" joins).
> >>>>>>>>>>>>>>>>>          > If
> >>>>>>>>>>>>>>>>>          > >>>>>>>> the
> >>>>>>>>>>>>>>>>>          > >>>>>>>> result key is the same key as key of
> >>>>> table1,
> >>>>>>> this
> >>>>>>>>>>>>>>>>>          implies that the
> >>>>>>>>>>>>>>>>>          > >>>>>>>> result can either be <A, join(X,1)>
> or
> >>> <A,
> >>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>          but not
> >>>>>>>>>>>>>>>>>          > both.
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Because the share the same key,
> >>> whatever
> >>>>>>> result
> >>>>>>>>>> record
> >>>>>>>>>>>>>>>>>          we emit
> >>>>>>>>>>>>>>>>>          > later,
> >>>>>>>>>>>>>>>>>          > >>>>>>>> overwrite the previous result.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> This is the reason why Jan originally
> >>>>> proposed
> >>>>>>>>>> to use
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>          > combination
> >>>>>>>>>>>>>>>>>          > >>>>>>>> of
> >>>>>>>>>>>>>>>>>          > >>>>>>>> both primary keys of the input tables
> >>> as
> >>>>> key
> >>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          output table.
> >>>>>>>>>>>>>>>>>          > >>>>>>>> This
> >>>>>>>>>>>>>>>>>          > >>>>>>>> makes the keys of the output table
> >>> unique
> >>>>> and
> >>>>>>> we
> >>>>>>>>>> can
> >>>>>>>>>>>>>>>>>          store both in
> >>>>>>>>>>>>>>>>>          > >>>>>>>> the
> >>>>>>>>>>>>>>>>>          > >>>>>>>> output table:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Result would be <A-a, join(X,1)>,
> <A-b,
> >>>>>>>>>> join(X,2)>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak
> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>> Just on remark here.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> The high-watermark could be
> >>> disregarded.
> >>>>> The
> >>>>>>>>>> decision
> >>>>>>>>>>>>>>>>>          about the
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> forward
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> depends on the size of the
> aggregated
> >>>>> map.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> Only 1 element long maps would be
> >>>>> unpacked
> >>>>>>> and
> >>>>>>>>>>>>>>>>>          forwarded. 0
> >>>>>>>>>>>>>>>>>          > element
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> maps
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> would be published as delete. Any
> >>> other
> >>>>> count
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> of map entries is in "waiting for
> >>> correct
> >>>>>>>>>> deletes to
> >>>>>>>>>>>>>>>>>          > arrive"-state.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
> >>>>> wrote:
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> It does look like I could replace
> the
> >>>>> second
> >>>>>>>>>>>>>>>>>          repartition store
> >>>>>>>>>>>>>>>>>          > and
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> highwater store with a groupBy and
> >>>>> reduce.
> >>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>          it looks
> >>>>>>>>>>>>>>>>>          > like
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> still need to store the highwater
> >>> value
> >>>>>>> within
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          materialized
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>>
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> compare the arrival of out-of-order
> >>>>> records
> >>>>>>>>>> (assuming
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          > >>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> of
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> THIS is correct...). This in effect
> is
> >>>>> the
> >>>>>>> same
> >>>>>>>>>> as
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>          design I
> >>>>>>>>>>>>>>>>>          > have
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> now,
> >>>>>>>>>>>>>>>>>          > >>>>>>>>> just with the two tables merged
> >>> together.
> >>>>>>>>>>>>>>>>>          > >>>>>>>>>
> >>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> --
> >>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>


-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
If they don't find the time:
They usually take the opposite path from me :D
so the answer would be clear.

hence my suggestion to vote.


On 04.12.2018 21:06, Adam Bellemare wrote:
> Hi Guozhang and Matthias
>
> I know both of you are quite busy, but we've gotten this KIP to a point
> where we need more guidance on the API (perhaps a bit of a tie-breaker, if
> you will). If you have anyone else you may think should look at this,
> please tag them accordingly.
>
> The scenario is as such:
>
> Current Option:
> API:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
> 1) Rekey the data to CombinedKey, and shuffles it to the partition with the
> foreignKey (repartition 1)
> 2) Join the data
> 3) Shuffle the data back to the original node (repartition 2)
> 4) Resolve out-of-order arrival / race condition due to foreign-key changes.
>
> Alternate Option:
> Perform #1 and #2 above, and return a KScatteredTable.
> - It would be keyed on a wrapped key function: <CombinedKey<KO, K>, VR> (KO
> = Other Table Key, K = This Table Key, VR = Joined Result)
> - KScatteredTable.resolve() would perform #3 and #4 but otherwise a user
> would be able to perform additional functions directly from the
> KScatteredTable (TBD - currently out of scope).
> - John's analysis 2-emails up is accurate as to the tradeoffs.
>
> Current Option is coded as-is. Alternate option is possible, but will
> require for implementation details to be made in the API and some exposure
> of new data structures into the API (ie: CombinedKey).
>
> I appreciate any insight into this.
>
> Thanks.
>
> On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi John
>>
>> Thanks for your feedback and assistance. I think your summary is accurate
>> from my perspective. Additionally, I would like to add that there is a risk
>> of inconsistent final states without performing the resolution. This is a
>> major concern for me as most of the data I have dealt with is produced by
>> relational databases. We have seen a number of cases where a user in the
>> Rails UI has modified the field (foreign key), realized they made a
>> mistake, and then updated the field again with a new key. The events are
>> propagated out as they are produced, and as such we have had real-world
>> cases where these inconsistencies were propagated downstream as the final
>> values due to the race conditions in the fanout of the data.
>>
>> This solution that I propose values correctness of the final result over
>> other factors.
>>
>> We could always move this function over to using a KScatteredTable
>> implementation in the future, and simply deprecate it this join API in
>> time. I think I would like to hear more from some of the other major
>> committers on which course of action they would think is best before any
>> more coding is done.
>>
>> Thanks again
>>
>> Adam
>>
>>
>> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io> wrote:
>>
>>> Hi Jan and Adam,
>>>
>>> Wow, thanks for doing that test, Adam. Those results are encouraging.
>>>
>>> Thanks for your performance experience as well, Jan. I agree that avoiding
>>> unnecessary join outputs is especially important when the fan-out is so
>>> high. I suppose this could also be built into the implementation we're
>>> discussing, but it wouldn't have to be specified in the KIP (since it's an
>>> API-transparent optimization).
>>>
>>> As far as whether or not to re-repartition the data, I didn't bring it up
>>> because it sounded like the two of you agreed to leave the KIP as-is,
>>> despite the disagreement.
>>>
>>> If you want my opinion, I feel like both approaches are reasonable.
>>> It sounds like Jan values more the potential for developers to optimize
>>> their topologies to re-use the intermediate nodes, whereas Adam places
>>> more
>>> value on having a single operator that people can use without extra steps
>>> at the end.
>>>
>>> Personally, although I do find it exceptionally annoying when a framework
>>> gets in my way when I'm trying to optimize something, it seems better to
>>> go
>>> for a single operation.
>>> * Encapsulating the internal transitions gives us significant latitude in
>>> the implementation (for example, joining only at the end, not in the
>>> middle
>>> to avoid extra data copying and out-of-order resolution; how we represent
>>> the first repartition keys (combined keys vs. value vectors), etc.). If we
>>> publish something like a KScatteredTable with the right-partitioned joined
>>> data, then the API pretty much locks in the implementation as well.
>>> * The API seems simpler to understand and use. I do mean "seems"; if
>>> anyone
>>> wants to make the case that KScatteredTable is actually simpler, I think
>>> hypothetical usage code would help. From a relational algebra perspective,
>>> it seems like KTable.join(KTable) should produce a new KTable in all
>>> cases.
>>> * That said, there might still be room in the API for a different
>>> operation
>>> like what Jan has proposed to scatter a KTable, and then do things like
>>> join, re-group, etc from there... I'm not sure; I haven't thought through
>>> all the consequences yet.
>>>
>>> This is all just my opinion after thinking over the discussion so far...
>>> -John
>>>
>>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <ad...@gmail.com>
>>> wrote:
>>>
>>>> Updated the PR to take into account John's feedback.
>>>>
>>>> I did some preliminary testing for the performance of the prefixScan. I
>>>> have attached the file, but I will also include the text in the body
>>> here
>>>> for archival purposes (I am not sure what happens to attached files). I
>>>> also updated the PR and the KIP accordingly.
>>>>
>>>> Summary: It scales exceptionally well for scanning large values of
>>>> records. As Jan mentioned previously, the real issue would be more
>>> around
>>>> processing the resulting records after obtaining them. For instance, it
>>>> takes approximately ~80-120 mS to flush the buffer and a further
>>> ~35-85mS
>>>> to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
>>>> through the records just to generate a simple count takes ~ 40 times
>>> longer
>>>> than the flush + scan combined.
>>>>
>>>>
>>> ============================================================================================
>>>> Setup:
>>>>
>>>>
>>> ============================================================================================
>>>> Java 9 with default settings aside from a 512 MB heap (Xmx512m, Xms512m)
>>>> CPU: i7 2.2 Ghz.
>>>>
>>>> Note: I am using a slightly-modified, directly-accessible Kafka Streams
>>>> RocksDB
>>>> implementation (RocksDB.java, basically just avoiding the
>>>> ProcessorContext).
>>>> There are no modifications to the default RocksDB values provided in the
>>>> 2.1/trunk release.
>>>>
>>>>
>>>> keysize = 128 bytes
>>>> valsize = 512 bytes
>>>>
>>>> Step 1:
>>>> Write X positive matching events: (key = prefix + left-padded
>>>> auto-incrementing integer)
>>>> Step 2:
>>>> Write 10X negative matching events (key = left-padded auto-incrementing
>>>> integer)
>>>> Step 3:
>>>> Perform flush
>>>> Step 4:
>>>> Perform prefixScan
>>>> Step 5:
>>>> Iterate through return Iterator and validate the count of expected
>>> events.
>>>>
>>>>
>>>>
>>> ============================================================================================
>>>> Results:
>>>>
>>>>
>>> ============================================================================================
>>>> X = 1k (11k events total)
>>>> Flush Time = 39 mS
>>>> Scan Time = 7 mS
>>>> 6.9 MB disk
>>>>
>>>>
>>> --------------------------------------------------------------------------------------------
>>>> X = 10k (110k events total)
>>>> Flush Time = 45 mS
>>>> Scan Time = 8 mS
>>>> 127 MB
>>>>
>>>>
>>> --------------------------------------------------------------------------------------------
>>>> X = 100k (1.1M events total)
>>>> Test1:
>>>> Flush Time = 60 mS
>>>> Scan Time = 12 mS
>>>> 678 MB
>>>>
>>>> Test2:
>>>> Flush Time = 45 mS
>>>> Scan Time = 7 mS
>>>> 576 MB
>>>>
>>>>
>>> --------------------------------------------------------------------------------------------
>>>> X = 1MB (11M events total)
>>>> Test1:
>>>> Flush Time = 52 mS
>>>> Scan Time = 19 mS
>>>> 7.2 GB
>>>>
>>>> Test2:
>>>> Flush Time = 84 mS
>>>> Scan Time = 34 mS
>>>> 9.1 GB
>>>>
>>>>
>>> --------------------------------------------------------------------------------------------
>>>> X = 2.5M (27.5M events total)
>>>> Test1:
>>>> Flush Time = 82 mS
>>>> Scan Time = 63 mS
>>>> 17GB - 276 sst files
>>>>
>>>> Test2:
>>>> Flush Time = 116 mS
>>>> Scan Time = 35 mS
>>>> 23GB - 361 sst files
>>>>
>>>> Test3:
>>>> Flush Time = 103 mS
>>>> Scan Time = 82 mS
>>>> 19 GB - 300 sst files
>>>>
>>>>
>>> --------------------------------------------------------------------------------------------
>>>>
>>>> I had to limit my testing on my laptop to X = 2.5M events. I tried to go
>>>> to X = 10M (110M events) but RocksDB was going into the 100GB+ range
>>> and my
>>>> laptop ran out of disk. More extensive testing could be done but I
>>> suspect
>>>> that it would be in line with what we're seeing in the results above.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> At this point in time, I think the only major discussion point is really
>>>> around what Jan and I have disagreed on: repartitioning back + resolving
>>>> potential out of order issues or leaving that up to the client to
>>> handle.
>>>>
>>>>
>>>> Thanks folks,
>>>>
>>>> Adam
>>>>
>>>>
>>>> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>>
>>>>>
>>>>> On 29.11.2018 15:14, John Roesler wrote:
>>>>>> Hi all,
>>>>>>
>>>>>> Sorry that this discussion petered out... I think the 2.1 release
>>>>> caused an
>>>>>> extended distraction that pushed it off everyone's radar (which was
>>>>>> precisely Adam's concern). Personally, I've also had some extend
>>>>>> distractions of my own that kept (and continue to keep) me
>>> preoccupied.
>>>>>>
>>>>>> However, calling for a vote did wake me up, so I guess Jan was on the
>>>>> right
>>>>>> track!
>>>>>>
>>>>>> I've gone back and reviewed the whole KIP document and the prior
>>>>>> discussion, and I'd like to offer a few thoughts:
>>>>>>
>>>>>> API Thoughts:
>>>>>>
>>>>>> 1. If I read the KIP right, you are proposing a many-to-one join.
>>> Could
>>>>> we
>>>>>> consider naming it manyToOneJoin? Or, if you prefer, flip the design
>>>>> around
>>>>>> and make it a oneToManyJoin?
>>>>>>
>>>>>> The proposed name "joinOnForeignKey" disguises the join type, and it
>>>>> seems
>>>>>> like it might trick some people into using it for a one-to-one join.
>>>>> This
>>>>>> would work, of course, but it would be super inefficient compared to
>>> a
>>>>>> simple rekey-and-join.
>>>>>>
>>>>>> 2. I might have missed it, but I don't think it's specified whether
>>>>> it's an
>>>>>> inner, outer, or left join. I'm guessing an outer join, as
>>> (neglecting
>>>>> IQ),
>>>>>> the rest can be achieved by filtering or by handling it in the
>>>>> ValueJoiner.
>>>>>>
>>>>>> 3. The arg list to joinOnForeignKey doesn't look quite right.
>>>>>> 3a. Regarding Serialized: There are a few different paradigms in
>>> play in
>>>>>> the Streams API, so it's confusing, but instead of three Serialized
>>>>> args, I
>>>>>> think it would be better to have one that allows (optionally) setting
>>>>> the 4
>>>>>> incoming serdes. The result serde is defined by the Materialized. The
>>>>>> incoming serdes can be optional because they might already be
>>> available
>>>>> on
>>>>>> the source KTables, or the default serdes from the config might be
>>>>>> applicable.
>>>>>>
>>>>>> 3b. Is the StreamPartitioner necessary? The other joins don't allow
>>>>> setting
>>>>>> one, and it seems like it might actually be harmful, since the rekey
>>>>>> operation needs to produce results that are co-partitioned with the
>>>>> "other"
>>>>>> KTable.
>>>>>>
>>>>>> 4. I'm fine with the "reserved word" header, but I didn't actually
>>>>> follow
>>>>>> what Matthias meant about namespacing requiring "deserializing" the
>>>>> record
>>>>>> header. The headers are already Strings, so I don't think that
>>>>>> deserialization is required. If we applied the namespace at source
>>> nodes
>>>>>> and stripped it at sink nodes, this would be practically no overhead.
>>>>> The
>>>>>> advantage of the namespace idea is that no public API change wrt
>>> headers
>>>>>> needs to happen, and no restrictions need to be placed on users'
>>>>> headers.
>>>>>>
>>>>>> (Although I'm wondering if we can get away without the header at
>>> all...
>>>>>> stay tuned)
>>>>>>
>>>>>> 5. I also didn't follow the discussion about the HWM table growing
>>>>> without
>>>>>> bound. As I read it, the HWM table is effectively implementing OCC to
>>>>>> resolve the problem you noted with disordering when the rekey is
>>>>>> reversed... particularly notable when the FK changes. As such, it
>>> only
>>>>>> needs to track the most recent "version" (the offset in the source
>>>>>> partition) of each key. Therefore, it should have the same number of
>>>>> keys
>>>>>> as the source table at all times.
>>>>>>
>>>>>> I see that you are aware of KIP-258, which I think might be relevant
>>> in
>>>>> a
>>>>>> couple of ways. One: it's just about storing the timestamp in the
>>> state
>>>>>> store, but the ultimate idea is to effectively use the timestamp as
>>> an
>>>>> OCC
>>>>>> "version" to drop disordered updates. You wouldn't want to use the
>>>>>> timestamp for this operation, but if you were to use a similar
>>>>> mechanism to
>>>>>> store the source offset in the store alongside the re-keyed values,
>>> then
>>>>>> you could avoid a separate table.
>>>>>>
>>>>>> 6. You and Jan have been thinking about this for a long time, so I've
>>>>>> probably missed something here, but I'm wondering if we can avoid the
>>>>> HWM
>>>>>> tracking at all and resolve out-of-order during a final join
>>> instead...
>>>>>>
>>>>>> Let's say we're joining a left table (Integer K: Letter FK, (other
>>>>> data))
>>>>>> to a right table (Letter K: (some data)).
>>>>>>
>>>>>> Left table:
>>>>>> 1: (A, xyz)
>>>>>> 2: (B, asd)
>>>>>>
>>>>>> Right table:
>>>>>> A: EntityA
>>>>>> B: EntityB
>>>>>>
>>>>>> We could do a rekey as you proposed with a combined key, but not
>>>>>> propagating the value at all..
>>>>>> Rekey table:
>>>>>> A-1: (dummy value)
>>>>>> B-2: (dummy value)
>>>>>>
>>>>>> Which we then join with the right table to produce:
>>>>>> A-1: EntityA
>>>>>> B-2: EntityB
>>>>>>
>>>>>> Which gets rekeyed back:
>>>>>> 1: A, EntityA
>>>>>> 2: B, EntityB
>>>>>>
>>>>>> And finally we do the actual join:
>>>>>> Result table:
>>>>>> 1: ((A, xyz), EntityA)
>>>>>> 2: ((B, asd), EntityB)
>>>>>>
>>>>>> The thing is that in that last join, we have the opportunity to
>>> compare
>>>>> the
>>>>>> current FK in the left table with the incoming PK of the right
>>> table. If
>>>>>> they don't match, we just drop the event, since it must be outdated.
>>>>>>
>>>>>
>>>>>> In your KIP, you gave an example in which (1: A, xyz) gets updated to
>>>>> (1:
>>>>>> B, xyz), ultimately yielding a conundrum about whether the final
>>> state
>>>>>> should be (1: null) or (1: joined-on-B). With the algorithm above,
>>> you
>>>>>> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
>>>>>> EntityB)). It seems like this does give you enough information to
>>> make
>>>>> the
>>>>>> right choice, regardless of disordering.
>>>>>
>>>>> Will check Adams patch, but this should work. As mentioned often I am
>>>>> not convinced on partitioning back for the user automatically. I think
>>>>> this is the real performance eater ;)
>>>>>
>>>>>>
>>>>>>
>>>>>> 7. Last thought... I'm a little concerned about the performance of
>>> the
>>>>>> range scans when records change in the right table. You've said that
>>>>> you've
>>>>>> been using the algorithm you presented in production for a while. Can
>>>>> you
>>>>>> give us a sense of the performance characteristics you've observed?
>>>>>>
>>>>>
>>>>> Make it work, make it fast, make it beautiful. The topmost thing here
>>> is
>>>>> / was correctness. In practice I do not measure the performance of the
>>>>> range scan. Usual cases I run this with is emitting 500k - 1kk rows
>>>>> on a left hand side change. The range scan is just the work you gotta
>>>>> do, also when you pack your data into different formats, usually the
>>>>> rocks performance is very tight to the size of the data and we can't
>>>>> really change that. It is more important for users to prevent useless
>>>>> updates to begin with. My left hand side is guarded to drop changes
>>> that
>>>>> are not going to change my join output.
>>>>>
>>>>> usually it's:
>>>>>
>>>>> drop unused fields and then don't forward if old.equals(new)
>>>>>
>>>>> regarding to the performance of creating an iterator for smaller
>>>>> fanouts, users can still just do a group by first then anyways.
>>>>>
>>>>>
>>>>>
>>>>>> I could only think of one alternative, but I'm not sure if it's
>>> better
>>>>> or
>>>>>> worse... If the first re-key only needs to preserve the original key,
>>>>> as I
>>>>>> proposed in #6, then we could store a vector of keys in the value:
>>>>>>
>>>>>> Left table:
>>>>>> 1: A,...
>>>>>> 2: B,...
>>>>>> 3: A,...
>>>>>>
>>>>>> Gets re-keyed:
>>>>>> A: [1, 3]
>>>>>> B: [2]
>>>>>>
>>>>>> Then, the rhs part of the join would only need a regular single-key
>>>>> lookup.
>>>>>> Of course we have to deal with the problem of large values, as
>>> there's
>>>>> no
>>>>>> bound on the number of lhs records that can reference rhs records.
>>>>> Offhand,
>>>>>> I'd say we could page the values, so when one row is past the
>>>>> threshold, we
>>>>>> append the key for the next page. Then in most cases, it would be a
>>>>> single
>>>>>> key lookup, but for large fan-out updates, it would be one per (max
>>>>> value
>>>>>> size)/(avg lhs key size).
>>>>>>
>>>>>> This seems more complex, though... Plus, I think there's some extra
>>>>>> tracking we'd need to do to know when to emit a retraction. For
>>> example,
>>>>>> when record 1 is deleted, the re-key table would just have (A: [3]).
>>>>> Some
>>>>>> kind of tombstone is needed so that the join result for 1 can also be
>>>>>> retracted.
>>>>>>
>>>>>> That's all!
>>>>>>
>>>>>> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
>>>>>> discussion has been slow.
>>>>>> -John
>>>>>>
>>>>>>
>>>>>> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>>> Jan.Filipiak@trivago.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Id say you can just call the vote.
>>>>>>>
>>>>>>> that happens all the time, and if something comes up, it just goes
>>> back
>>>>>>> to discuss.
>>>>>>>
>>>>>>> would not expect to much attention with another another email in
>>> this
>>>>>>> thread.
>>>>>>>
>>>>>>> best Jan
>>>>>>>
>>>>>>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>>>>>>> Hello Contributors
>>>>>>>>
>>>>>>>> I know that 2.1 is about to be released, but I do need to bump
>>> this to
>>>>>>> keep
>>>>>>>> visibility up. I am still intending to push this through once
>>>>> contributor
>>>>>>>> feedback is given.
>>>>>>>>
>>>>>>>> Main points that need addressing:
>>>>>>>> 1) Any way (or benefit) in structuring the current singular graph
>>> node
>>>>>>> into
>>>>>>>> multiple nodes? It has a whopping 25 parameters right now. I am a
>>> bit
>>>>>>> fuzzy
>>>>>>>> on how the optimizations are supposed to work, so I would
>>> appreciate
>>>>> any
>>>>>>>> help on this aspect.
>>>>>>>>
>>>>>>>> 2) Overall strategy for joining + resolving. This thread has much
>>>>>>> discourse
>>>>>>>> between Jan and I between the current highwater mark proposal and a
>>>>>>> groupBy
>>>>>>>> + reduce proposal. I am of the opinion that we need to strictly
>>> handle
>>>>>>> any
>>>>>>>> chance of out-of-order data and leave none of it up to the
>>> consumer.
>>>>> Any
>>>>>>>> comments or suggestions here would also help.
>>>>>>>>
>>>>>>>> 3) Anything else that you see that would prevent this from moving
>>> to a
>>>>>>> vote?
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>>
>>>>>>>> Adam
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>>>>>>> adam.bellemare@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Jan
>>>>>>>>>
>>>>>>>>> With the Stores.windowStoreBuilder and
>>> Stores.persistentWindowStore,
>>>>> you
>>>>>>>>> actually only need to specify the amount of segments you want and
>>> how
>>>>>>> large
>>>>>>>>> they are. To the best of my understanding, what happens is that
>>> the
>>>>>>>>> segments are automatically rolled over as new data with new
>>>>> timestamps
>>>>>>> are
>>>>>>>>> created. We use this exact functionality in some of the work done
>>>>>>>>> internally at my company. For reference, this is the hopping
>>> windowed
>>>>>>> store.
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>>>>>>
>>>>>>>>> In the code that I have provided, there are going to be two 24h
>>>>>>> segments.
>>>>>>>>> When a record is put into the windowStore, it will be inserted at
>>>>> time
>>>>>>> T in
>>>>>>>>> both segments. The two segments will always overlap by 12h. As
>>> time
>>>>>>> goes on
>>>>>>>>> and new records are added (say at time T+12h+), the oldest segment
>>>>> will
>>>>>>> be
>>>>>>>>> automatically deleted and a new segment created. The records are
>>> by
>>>>>>> default
>>>>>>>>> inserted with the context.timestamp(), such that it is the record
>>>>> time,
>>>>>>> not
>>>>>>>>> the clock time, which is used.
>>>>>>>>>
>>>>>>>>> To the best of my understanding, the timestamps are retained when
>>>>>>>>> restoring from the changelog.
>>>>>>>>>
>>>>>>>>> Basically, this is heavy-handed way to deal with TTL at a
>>>>> segment-level,
>>>>>>>>> instead of at an individual record level.
>>>>>>>>>
>>>>>>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>>>>> Jan.Filipiak@trivago.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Will that work? I expected it to blow up with ClassCastException
>>> or
>>>>>>>>>> similar.
>>>>>>>>>>
>>>>>>>>>> You either would have to specify the window you fetch/put or
>>> iterate
>>>>>>>>>> across all windows the key was found in right?
>>>>>>>>>>
>>>>>>>>>> I just hope the window-store doesn't check stream-time under the
>>>>> hoods
>>>>>>>>>> that would be a questionable interface.
>>>>>>>>>>
>>>>>>>>>> If it does: did you see my comment on checking all the windows
>>>>> earlier?
>>>>>>>>>> that would be needed to actually give reasonable time gurantees.
>>>>>>>>>>
>>>>>>>>>> Best
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>>>>>>> Hi Jan
>>>>>>>>>>>
>>>>>>>>>>> Check for  " highwaterMat " in the PR. I only changed the state
>>>>> store,
>>>>>>>>>> not
>>>>>>>>>>> the ProcessorSupplier.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Adam
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the information. This is indeed something that
>>> will be
>>>>>>>>>>>>> extremely
>>>>>>>>>>>>> useful for this KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>> Thanks for your explanations. That being said, I will not be
>>>>> moving
>>>>>>>>>> ahead
>>>>>>>>>>>>> with an implementation using reshuffle/groupBy solution as you
>>>>>>>>>> propose.
>>>>>>>>>>>>> That being said, if you wish to implement it yourself off of
>>> my
>>>>>>>>>> current PR
>>>>>>>>>>>>> and submit it as a competitive alternative, I would be more
>>> than
>>>>>>>>>> happy to
>>>>>>>>>>>>> help vet that as an alternate solution. As it stands right
>>> now,
>>>>> I do
>>>>>>>>>> not
>>>>>>>>>>>>> really have more time to invest into alternatives without
>>> there
>>>>>>> being
>>>>>>>>>> a
>>>>>>>>>>>>> strong indication from the binding voters which they would
>>>>> prefer.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> Hey, total no worries. I think I personally gave up on the
>>> streams
>>>>>>> DSL
>>>>>>>>>> for
>>>>>>>>>>>> some time already, otherwise I would have pulled this KIP
>>> through
>>>>>>>>>> already.
>>>>>>>>>>>> I am currently reimplementing my own DSL based on PAPI.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> I will look at finishing up my PR with the windowed state
>>> store
>>>>> in
>>>>>>> the
>>>>>>>>>>>>> next
>>>>>>>>>>>>> week or so, exercising it via tests, and then I will come back
>>>>> for
>>>>>>>>>> final
>>>>>>>>>>>>> discussions. In the meantime, I hope that any of the binding
>>>>> voters
>>>>>>>>>> could
>>>>>>>>>>>>> take a look at the KIP in the wiki. I have updated it
>>> according
>>>>> to
>>>>>>> the
>>>>>>>>>>>>> latest plan:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have also updated the KIP PR to use a windowed store. This
>>>>> could
>>>>>>> be
>>>>>>>>>>>>> replaced by the results of KIP-258 whenever they are
>>> completed.
>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated
>>> in
>>>>> the
>>>>>>>>>> PR?
>>>>>>>>>>>> expected it to change to Windowed<K>,Long Missing something?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link,
>>>>> as it
>>>>>>>>>> is
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258
>>> we do
>>>>>>>>>> want to
>>>>>>>>>>>>>> have "handling out-of-order data for source KTable" such that
>>>>>>>>>> instead of
>>>>>>>>>>>>>> blindly apply the updates to the materialized store, i.e.
>>>>> following
>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>> ordering, we will reject updates that are older than the
>>> current
>>>>>>>>>> key's
>>>>>>>>>>>>>> timestamps, i.e. following timestamp ordering.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
>>> the
>>>>>>> high
>>>>>>>>>>>>>>> watermark store, now altered to be replaced with a window
>>>>> store),
>>>>>>> I
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>> another current on-going KIP may actually help:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This is for adding the timestamp into a key-value store
>>> (i.e.
>>>>> only
>>>>>>>>>> for
>>>>>>>>>>>>>>> non-windowed KTable), and then one of its usage, as
>>> described
>>>>> in
>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that
>>> we
>>>>> can
>>>>>>>>>> then
>>>>>>>>>>>>>>> "reject" updates from the source topics if its timestamp is
>>>>>>> smaller
>>>>>>>>>> than
>>>>>>>>>>>>>>> the current key's latest update timestamp. I think it is
>>> very
>>>>>>>>>> similar to
>>>>>>>>>>>>>>> what you have in mind for high watermark based filtering,
>>> while
>>>>>>> you
>>>>>>>>>> only
>>>>>>>>>>>>>>> need to make sure that the timestamps of the joining records
>>>>> are
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> correctly
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> inherited though the whole topology to the final stage.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Note that this KIP is for key-value store and hence
>>>>> non-windowed
>>>>>>>>>> KTables
>>>>>>>>>>>>>>> only, but for windowed KTables we do not really have a good
>>>>>>> support
>>>>>>>>>> for
>>>>>>>>>>>>>>> their joins anyways (
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>> think we can just consider non-windowed KTable-KTable
>>> non-key
>>>>>>> joins
>>>>>>>>>> for
>>>>>>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Current highwater mark implementation would grow endlessly
>>>>> based
>>>>>>>>>> on
>>>>>>>>>>>>>>>>> primary key of original event. It is a pair of (<this
>>> table
>>>>>>>>>> primary
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> key>,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> <highest offset seen for that key>). This is used to
>>>>> differentiate
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> late arrivals and new updates. My newest proposal would be
>>> to
>>>>>>>>>> replace
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> with a Windowed state store of Duration N. This would allow
>>> the
>>>>>>> same
>>>>>>>>>>>>>>>>> behaviour, but cap the size based on time. This should
>>> allow
>>>>> for
>>>>>>>>>> all
>>>>>>>>>>>>>>>>> late-arriving events to be processed, and should be
>>>>> customizable
>>>>>>>>>> by
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
>>>>> minutes
>>>>>>> of
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> window,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Adam, using time based retention can do the trick here.
>>>>> Even
>>>>>>>>>> if I
>>>>>>>>>>>>>>>> would still like to see the automatic repartitioning
>>> optional
>>>>>>>>>> since I
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> just reshuffle again. With windowed store I am a little bit
>>>>>>>>>> sceptical
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> how to determine the window. So esentially one could run
>>> into
>>>>>>>>>> problems
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> the rapid change happens near a window border. I will check
>>> you
>>>>>>>>>>>>>>>> implementation in detail, if its problematic, we could
>>> still
>>>>>>> check
>>>>>>>>>>>>>>>> _all_
>>>>>>>>>>>>>>>> windows on read with not to bad performance impact I guess.
>>>>> Will
>>>>>>>>>> let
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>> know if the implementation would be correct as is. I
>>> wouldn't
>>>>> not
>>>>>>>>>> like
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>>>>>>> timestamp(B).
>>>>>>>>>> I
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> we can't expect that.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>>> I believe I understand what you mean now - thanks for the
>>>>>>>>>> diagram, it
>>>>>>>>>>>>>>>>> did really help. You are correct that I do not have the
>>>>> original
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> primary
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> key available, and I can see that if it was available then
>>> you
>>>>>>>>>> would be
>>>>>>>>>>>>>>>>> able to add and remove events from the Map. That being
>>> said,
>>>>> I
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> encourage
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> you to finish your diagrams / charts just for clarity for
>>>>> everyone
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> else.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But
>>> I
>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> the benefits for the rest. Sorry about the original primary
>>>>> key,
>>>>>>> We
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>> join and Group by implemented our own in PAPI and basically
>>>>> not
>>>>>>>>>> using
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> DSL (Just the abstraction). Completely missed that in
>>> original
>>>>> DSL
>>>>>>>>>> its
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> there and just assumed it. total brain mess up on my end.
>>> Will
>>>>>>>>>> finish
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> chart as soon as i get a quite evening this week.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> My follow up question for you is, won't the Map stay inside
>>>>> the
>>>>>>>>>> State
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Store indefinitely after all of the changes have
>>> propagated?
>>>>>>> Isn't
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> effectively the same as a highwater mark state store?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
>>>>> return
>>>>>>>>>> `null`
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> the key is removed from the keyspace. But there is going to
>>> be
>>>>> a
>>>>>>>>>> store
>>>>>>>>>>>>>>>> 100%, the good thing is that I can use this store directly
>>> for
>>>>>>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
>>> store,
>>>>>>>>>> satisfying
>>>>>>>>>>>>>>>> all gurantees needed for further groupby / join. The
>>> Windowed
>>>>>>>>>> store is
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> keeping the values, so for the next statefull operation we
>>>>> would
>>>>>>>>>>>>>>>> need to instantiate an extra store. or we have the window
>>>>> store
>>>>>>>>>> also
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> the values then.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Long story short. if we can flip in a custom group by
>>> before
>>>>>>>>>>>>>>>> repartitioning to the original primary key i think it would
>>>>> help
>>>>>>>>>> the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> big time in building efficient apps. Given the original
>>> primary
>>>>>>> key
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> understand that we do not have a solid foundation to build
>>> on.
>>>>>>>>>>>>>>>> Leaving primary key carry along to the user. very
>>>>> unfortunate. I
>>>>>>>>>> could
>>>>>>>>>>>>>>>> understand the decision goes like that. I do not think its
>>> a
>>>>> good
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> decision.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>>>>> dumbreprajakta311@gmail.com
>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          please remove me from this group
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > Hi Adam,
>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>          > give me some time, will make such a chart. last
>>>>> time i
>>>>>>>>>> didn't
>>>>>>>>>>>>>>>>>          get along
>>>>>>>>>>>>>>>>>          > well with giphy and ruined all your charts.
>>>>>>>>>>>>>>>>>          > Hopefully i can get it done today
>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>          > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>          > > Hi Jan
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > > I have included a diagram of what I attempted
>>> on
>>>>> the
>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>>>>>>>
>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>>>>>>          <
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>>>>>>>
>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > > I attempted this back at the start of my own
>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>          this
>>>>>>>>>>>>>>>>>          > > solution, and since I could not get it to
>>> work I
>>>>> have
>>>>>>>>>> since
>>>>>>>>>>>>>>>>>          discarded the
>>>>>>>>>>>>>>>>>          > > code. At this point in time, if you wish to
>>>>> continue
>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>          for your
>>>>>>>>>>>>>>>>>          > > groupBy solution, I ask that you please
>>> create a
>>>>>>>>>> diagram on
>>>>>>>>>>>>>>>>>          the KIP
>>>>>>>>>>>>>>>>>          > > carefully explaining your solution. Please
>>> feel
>>>>> free
>>>>>>> to
>>>>>>>>>> use
>>>>>>>>>>>>>>>>>          the image I
>>>>>>>>>>>>>>>>>          > > just posted as a starting point. I am having
>>>>> trouble
>>>>>>>>>>>>>>>>>          understanding your
>>>>>>>>>>>>>>>>>          > > explanations but I think that a carefully
>>>>> constructed
>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>          will clear
>>>>>>>>>>>>>>>>>          > up
>>>>>>>>>>>>>>>>>          > > any misunderstandings. Alternately, please
>>> post a
>>>>>>>>>>>>>>>>>          comprehensive PR with
>>>>>>>>>>>>>>>>>          > > your solution. I can only guess at what you
>>>>> mean, and
>>>>>>>>>> since I
>>>>>>>>>>>>>>>>>          value my
>>>>>>>>>>>>>>>>>          > own
>>>>>>>>>>>>>>>>>          > > time as much as you value yours, I believe it
>>> is
>>>>> your
>>>>>>>>>>>>>>>>>          responsibility to
>>>>>>>>>>>>>>>>>          > > provide an implementation instead of me
>>> trying to
>>>>>>> guess.
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > > Adam
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > > wrote:
>>>>>>>>>>>>>>>>>          > >
>>>>>>>>>>>>>>>>>          > >> Hi James,
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >> nice to see you beeing interested. kafka
>>>>> streams at
>>>>>>>>>> this
>>>>>>>>>>>>>>>>>          point supports
>>>>>>>>>>>>>>>>>          > >> all sorts of joins as long as both streams
>>> have
>>>>> the
>>>>>>>>>> same
>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>          > >> Adam is currently implementing a join where a
>>>>> KTable
>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>          KTable can
>>>>>>>>>>>>>>>>>          > have
>>>>>>>>>>>>>>>>>          > >> a one to many relation ship (1:n). We exploit
>>>>> that
>>>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > >> datastore that keeps data sorted (At least
>>>>> exposes an
>>>>>>>>>> API to
>>>>>>>>>>>>>>>>>          access the
>>>>>>>>>>>>>>>>>          > >> stored data in a sorted fashion).
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >> I think the technical caveats are well
>>>>> understood
>>>>>>> now
>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > basically
>>>>>>>>>>>>>>>>>          > >> down to philosophy and API Design ( when Adam
>>>>> sees
>>>>>>> my
>>>>>>>>>> newest
>>>>>>>>>>>>>>>>>          message).
>>>>>>>>>>>>>>>>>          > >> I have a lengthy track record of loosing
>>> those
>>>>> kinda
>>>>>>>>>>>>>>>>>          arguments within
>>>>>>>>>>>>>>>>>          > the
>>>>>>>>>>>>>>>>>          > >> streams community and I have no clue why. So
>>> I
>>>>>>>>>> literally
>>>>>>>>>>>>>>>>>          can't wait for
>>>>>>>>>>>>>>>>>          > you
>>>>>>>>>>>>>>>>>          > >> to churn through this thread and give you
>>>>> opinion on
>>>>>>>>>> how we
>>>>>>>>>>>>>>>>>          should
>>>>>>>>>>>>>>>>>          > design
>>>>>>>>>>>>>>>>>          > >> the return type of the oneToManyJoin and how
>>>>> many
>>>>>>>>>> power we
>>>>>>>>>>>>>>>>>          want to give
>>>>>>>>>>>>>>>>>          > to
>>>>>>>>>>>>>>>>>          > >> the user vs "simplicity" (where simplicity
>>> isn't
>>>>>>>>>> really that
>>>>>>>>>>>>>>>>>          as users
>>>>>>>>>>>>>>>>>          > still
>>>>>>>>>>>>>>>>>          > >> need to understand it I argue)
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >> waiting for you to join in on the discussion
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >> Best Jan
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>>>>>>>>>>>>          > >>
>>>>>>>>>>>>>>>>>          > >>> I am new to this group and I found this
>>> subject
>>>>>>>>>>>>>>>>>          interesting.  Sounds
>>>>>>>>>>>>>>>>>          > like
>>>>>>>>>>>>>>>>>          > >>> you guys want to implement a join table of
>>> two
>>>>>>>>>> streams? Is
>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>          > somewhere
>>>>>>>>>>>>>>>>>          > >>> I can see the original requirement or
>>> proposal?
>>>>>>>>>>>>>>>>>          > >>>
>>>>>>>>>>>>>>>>>          > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>>>>>>>>>>>>          <Jan.Filipiak@trivago.com <mailto:
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>> wrote:
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>> I'm currently testing using a Windowed
>>> Store
>>>>> to
>>>>>>>>>> store the
>>>>>>>>>>>>>>>>>          highwater
>>>>>>>>>>>>>>>>>          > >>>>> mark.
>>>>>>>>>>>>>>>>>          > >>>>> By all indications this should work fine,
>>>>> with
>>>>>>> the
>>>>>>>>>> caveat
>>>>>>>>>>>>>>>>>          being that
>>>>>>>>>>>>>>>>>          > it
>>>>>>>>>>>>>>>>>          > >>>>> can
>>>>>>>>>>>>>>>>>          > >>>>> only resolve out-of-order arrival for up
>>> to
>>>>> the
>>>>>>>>>> size of
>>>>>>>>>>>>>>>>>          the window
>>>>>>>>>>>>>>>>>          > (ie:
>>>>>>>>>>>>>>>>>          > >>>>> 24h, 72h, etc). This would remove the
>>>>> possibility
>>>>>>>>>> of it
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > unbounded
>>>>>>>>>>>>>>>>>          > >>>>> in
>>>>>>>>>>>>>>>>>          > >>>>> size.
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>> With regards to Jan's suggestion, I
>>> believe
>>>>> this
>>>>>>> is
>>>>>>>>>> where
>>>>>>>>>>>>>>>>>          we will
>>>>>>>>>>>>>>>>>          > have
>>>>>>>>>>>>>>>>>          > >>>>> to
>>>>>>>>>>>>>>>>>          > >>>>> remain in disagreement. While I do not
>>>>> disagree
>>>>>>>>>> with your
>>>>>>>>>>>>>>>>>          statement
>>>>>>>>>>>>>>>>>          > >>>>> about
>>>>>>>>>>>>>>>>>          > >>>>> there likely to be additional joins done
>>> in a
>>>>>>>>>> real-world
>>>>>>>>>>>>>>>>>          workflow, I
>>>>>>>>>>>>>>>>>          > do
>>>>>>>>>>>>>>>>>          > >>>>> not
>>>>>>>>>>>>>>>>>          > >>>>> see how you can conclusively deal with
>>>>>>> out-of-order
>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>          > >>>>> foreign-key
>>>>>>>>>>>>>>>>>          > >>>>> changes and subsequent joins. I have
>>>>> attempted
>>>>>>> what
>>>>>>>>>> I
>>>>>>>>>>>>>>>>>          think you have
>>>>>>>>>>>>>>>>>          > >>>>> proposed (without a high-water, using
>>>>> groupBy and
>>>>>>>>>> reduce)
>>>>>>>>>>>>>>>>>          and found
>>>>>>>>>>>>>>>>>          > >>>>> that if
>>>>>>>>>>>>>>>>>          > >>>>> the foreign key changes too quickly, or
>>> the
>>>>> load
>>>>>>> on
>>>>>>>>>> a
>>>>>>>>>>>>>>>>>          stream thread
>>>>>>>>>>>>>>>>>          > is
>>>>>>>>>>>>>>>>>          > >>>>> too
>>>>>>>>>>>>>>>>>          > >>>>> high, the joined messages will arrive
>>>>>>> out-of-order
>>>>>>>>>> and be
>>>>>>>>>>>>>>>>>          incorrectly
>>>>>>>>>>>>>>>>>          > >>>>> propagated, such that an intermediate
>>> event
>>>>> is
>>>>>>>>>>>>>>>>> represented
>>>>>>>>>>>>>>>>>          as the
>>>>>>>>>>>>>>>>>          > final
>>>>>>>>>>>>>>>>>          > >>>>> event.
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>> Can you shed some light on your groupBy
>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>          There must be
>>>>>>>>>>>>>>>>>          > >>>> some sort of flaw in it.
>>>>>>>>>>>>>>>>>          > >>>> I have a suspicion where it is, I would
>>> just
>>>>> like
>>>>>>> to
>>>>>>>>>>>>>>>>>          confirm. The idea
>>>>>>>>>>>>>>>>>          > >>>> is bullet proof and it must be
>>>>>>>>>>>>>>>>>          > >>>> an implementation mess up. I would like to
>>>>> clarify
>>>>>>>>>> before
>>>>>>>>>>>>>>>>>          we draw a
>>>>>>>>>>>>>>>>>          > >>>> conclusion.
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>    Repartitioning the scattered events
>>> back to
>>>>>>> their
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > >>>>> partitions is the only way I know how to
>>>>>>> conclusively
>>>>>>>>>> deal
>>>>>>>>>>>>>>>>>          with
>>>>>>>>>>>>>>>>>          > >>>>> out-of-order events in a given time frame,
>>>>> and to
>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>          that the
>>>>>>>>>>>>>>>>>          > data
>>>>>>>>>>>>>>>>>          > >>>>> is
>>>>>>>>>>>>>>>>>          > >>>>> eventually consistent with the input
>>> events.
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>> If you have some code to share that
>>>>> illustrates
>>>>>>> your
>>>>>>>>>>>>>>>>>          approach, I
>>>>>>>>>>>>>>>>>          > would
>>>>>>>>>>>>>>>>>          > >>>>> be
>>>>>>>>>>>>>>>>>          > >>>>> very grateful as it would remove any
>>>>>>>>>> misunderstandings
>>>>>>>>>>>>>>>>>          that I may
>>>>>>>>>>>>>>>>>          > have.
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>> ah okay you were looking for my code. I
>>> don't
>>>>> have
>>>>>>>>>>>>>>>>>          something easily
>>>>>>>>>>>>>>>>>          > >>>> readable here as its bloated with
>>> OO-patterns.
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>> its anyhow trivial:
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>> @Override
>>>>>>>>>>>>>>>>>          > >>>>      public T apply(K aggKey, V value, T
>>>>>>> aggregate)
>>>>>>>>>>>>>>>>>          > >>>>      {
>>>>>>>>>>>>>>>>>          > >>>>          Map<U, V> currentStateAsMap =
>>>>>>>>>> asMap(aggregate);
>>>>>>>>>>>>>>>>> <<
>>>>>>>>>>>>>>>>>          imaginary
>>>>>>>>>>>>>>>>>          > >>>>          U toModifyKey =
>>> mapper.apply(value);
>>>>>>>>>>>>>>>>>          > >>>>              << this is the place where
>>> people
>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>          gonna have
>>>>>>>>>>>>>>>>>          > issues
>>>>>>>>>>>>>>>>>          > >>>> and why you probably couldn't do it. we
>>> would
>>>>> need
>>>>>>>>>> to find
>>>>>>>>>>>>>>>>>          a solution
>>>>>>>>>>>>>>>>>          > here.
>>>>>>>>>>>>>>>>>          > >>>> I didn't realize that yet.
>>>>>>>>>>>>>>>>>          > >>>>              << we propagate the field in
>>> the
>>>>>>>>>> joiner, so
>>>>>>>>>>>>>>>>>          that we can
>>>>>>>>>>>>>>>>>          > pick
>>>>>>>>>>>>>>>>>          > >>>> it up in an aggregate. Probably you have
>>> not
>>>>>>> thought
>>>>>>>>>> of
>>>>>>>>>>>>>>>>>          this in your
>>>>>>>>>>>>>>>>>          > >>>> approach right?
>>>>>>>>>>>>>>>>>          > >>>>              << I am very open to find a
>>>>> generic
>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>          here. In my
>>>>>>>>>>>>>>>>>          > >>>> honest opinion this is broken in
>>>>>>> KTableImpl.GroupBy
>>>>>>>>>> that
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>          looses
>>>>>>>>>>>>>>>>>          > the keys
>>>>>>>>>>>>>>>>>          > >>>> and only maintains the aggregate key.
>>>>>>>>>>>>>>>>>          > >>>>              << I abstracted it away back
>>>>> then way
>>>>>>>>>> before
>>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>          > thinking
>>>>>>>>>>>>>>>>>          > >>>> of oneToMany join. That is why I didn't
>>>>> realize
>>>>>>> its
>>>>>>>>>>>>>>>>>          significance here.
>>>>>>>>>>>>>>>>>          > >>>>              << Opinions?
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>          for (V m : current)
>>>>>>>>>>>>>>>>>          > >>>>          {
>>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>>>>>>>>>>>>>          > >>>>          }
>>>>>>>>>>>>>>>>>          > >>>>          if (isAdder)
>>>>>>>>>>>>>>>>>          > >>>>          {
>>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>>>>>>>>>>>>>          > >>>>          }
>>>>>>>>>>>>>>>>>          > >>>>          else
>>>>>>>>>>>>>>>>>          > >>>>          {
>>>>>>>>>>>>>>>>>          > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>>>>>>          > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>>>>>>          > >>>>                  return null;
>>>>>>>>>>>>>>>>>          > >>>>              }
>>>>>>>>>>>>>>>>>          > >>>>          }
>>>>>>>>>>>>>>>>>          > >>>>          retrun
>>>>> asAggregateType(currentStateAsMap)
>>>>>>>>>>>>>>>>>          > >>>>      }
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>>
>>>>>>>>>>>>>>>>>          > >>>> Thanks,
>>>>>>>>>>>>>>>>>          > >>>>> Adam
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
>>> Filipiak
>>>>> <
>>>>>>>>>>>>>>>>>          > Jan.Filipiak@trivago.com <mailto:
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>> wrote:
>>>>>>>>>>>>>>>>>          > >>>>>
>>>>>>>>>>>>>>>>>          > >>>>> Thanks Adam for bringing Matthias to
>>> speed!
>>>>>>>>>>>>>>>>>          > >>>>>> about the differences. I think re-keying
>>>>> back
>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>          optional at
>>>>>>>>>>>>>>>>>          > >>>>>> best.
>>>>>>>>>>>>>>>>>          > >>>>>> I would say we return a KScatteredTable
>>> with
>>>>>>>>>> reshuffle()
>>>>>>>>>>>>>>>>>          returning
>>>>>>>>>>>>>>>>>          > >>>>>> KTable<originalKey,Joined> to make the
>>>>> backwards
>>>>>>>>>>>>>>>>>          repartitioning
>>>>>>>>>>>>>>>>>          > >>>>>> optional.
>>>>>>>>>>>>>>>>>          > >>>>>> I am also in a big favour of doing the
>>> out
>>>>> of
>>>>>>> order
>>>>>>>>>>>>>>>>>          processing using
>>>>>>>>>>>>>>>>>          > >>>>>> group
>>>>>>>>>>>>>>>>>          > >>>>>> by instead high water mark tracking.
>>>>>>>>>>>>>>>>>          > >>>>>> Just because unbounded growth is just
>>> scary
>>>>> + It
>>>>>>>>>> saves
>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>          the header
>>>>>>>>>>>>>>>>>          > >>>>>> stuff.
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>> I think the abstraction of always
>>>>> repartitioning
>>>>>>>>>> back is
>>>>>>>>>>>>>>>>>          just not so
>>>>>>>>>>>>>>>>>          > >>>>>> strong. Like the work has been done
>>> before
>>>>> we
>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>          back and
>>>>>>>>>>>>>>>>>          > >>>>>> grouping
>>>>>>>>>>>>>>>>>          > >>>>>> by something else afterwards is really
>>>>> common.
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>> On 05.09.2018 13:49, Adam Bellemare
>>> wrote:
>>>>>>>>>>>>>>>>>          > >>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>> Hi Matthias
>>>>>>>>>>>>>>>>>          > >>>>>>> Thank you for your feedback, I do
>>>>> appreciate
>>>>>>> it!
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> While name spacing would be possible, it
>>>>> would
>>>>>>>>>> require
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>          > deserialize
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> user headers what implies a runtime
>>>>> overhead.
>>>>>>> I
>>>>>>>>>> would
>>>>>>>>>>>>>>>>>          suggest to
>>>>>>>>>>>>>>>>>          > no
>>>>>>>>>>>>>>>>>          > >>>>>>>> namespace for now to avoid the
>>> overhead.
>>>>> If
>>>>>>> this
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > problem in
>>>>>>>>>>>>>>>>>          > >>>>>>>> the future, we can still add name
>>> spacing
>>>>>>> later
>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Agreed. I will go with using a reserved
>>>>> string
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>          document it.
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> My main concern about the design it the
>>>>> type of
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          result KTable:
>>>>>>>>>>>>>>>>>          > If
>>>>>>>>>>>>>>>>>          > >>>>>>> I
>>>>>>>>>>>>>>>>>          > >>>>>>> understood the proposal correctly,
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> In your example, you have table1 and
>>> table2
>>>>>>>>>> swapped.
>>>>>>>>>>>>>>>>>          Here is how it
>>>>>>>>>>>>>>>>>          > >>>>>>> works
>>>>>>>>>>>>>>>>>          > >>>>>>> currently:
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> 1) table1 has the records that contain
>>> the
>>>>>>>>>> foreign key
>>>>>>>>>>>>>>>>>          within their
>>>>>>>>>>>>>>>>>          > >>>>>>> value.
>>>>>>>>>>>>>>>>>          > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>          > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>>>>>>          > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> 2) A Value mapper is required to extract
>>>>> the
>>>>>>>>>> foreign
>>>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>          > >>>>>>> table1 foreign key mapper: ( value =>
>>>>> value.fk
>>>>>>>>>>>>>>>>>          <http://value.fk> )
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> The mapper is applied to each element in
>>>>>>> table1,
>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>          new combined
>>>>>>>>>>>>>>>>>          > >>>>>>> key is
>>>>>>>>>>>>>>>>>          > >>>>>>> made:
>>>>>>>>>>>>>>>>>          > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>,
>>> <A-b,
>>>>>>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>>>>>>          <B-c,
>>>>>>>>>>>>>>>>>          > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> 3) The rekeyed events are copartitioned
>>>>> with
>>>>>>>>>> table2:
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>>>>>>>>>>>>          > >>>>>>> RepartitionedTable1: <A-a,
>>> (fk=A,bar=1)>,
>>>>> <A-b,
>>>>>>>>>>>>>>>>>          (fk=A,bar=2)>
>>>>>>>>>>>>>>>>>          > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>>>>>>>>>>>>          > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>>>>>>>>>>>>          > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> 4) From here, they can be joined
>>> together
>>>>>>> locally
>>>>>>>>>> by
>>>>>>>>>>>>>>>>>          applying the
>>>>>>>>>>>>>>>>>          > >>>>>>> joiner
>>>>>>>>>>>>>>>>>          > >>>>>>> function.
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> At this point, Jan's design and my
>>> design
>>>>>>>>>> deviate. My
>>>>>>>>>>>>>>>>>          design goes
>>>>>>>>>>>>>>>>>          > on
>>>>>>>>>>>>>>>>>          > >>>>>>> to
>>>>>>>>>>>>>>>>>          > >>>>>>> repartition the data post-join and
>>> resolve
>>>>>>>>>> out-of-order
>>>>>>>>>>>>>>>>>          arrival of
>>>>>>>>>>>>>>>>>          > >>>>>>> records,
>>>>>>>>>>>>>>>>>          > >>>>>>> finally returning the data keyed just
>>> the
>>>>>>>>>> original key.
>>>>>>>>>>>>>>>>>          I do not
>>>>>>>>>>>>>>>>>          > >>>>>>> expose
>>>>>>>>>>>>>>>>>          > >>>>>>> the
>>>>>>>>>>>>>>>>>          > >>>>>>> CombinedKey or any of the internals
>>>>> outside of
>>>>>>> the
>>>>>>>>>>>>>>>>>          joinOnForeignKey
>>>>>>>>>>>>>>>>>          > >>>>>>> function. This does make for larger
>>>>> footprint,
>>>>>>>>>> but it
>>>>>>>>>>>>>>>>>          removes all
>>>>>>>>>>>>>>>>>          > >>>>>>> agency
>>>>>>>>>>>>>>>>>          > >>>>>>> for resolving out-of-order arrivals and
>>>>>>> handling
>>>>>>>>>>>>>>>>>          CombinedKeys from
>>>>>>>>>>>>>>>>>          > the
>>>>>>>>>>>>>>>>>          > >>>>>>> user. I believe that this makes the
>>>>> function
>>>>>>> much
>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>          to use.
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> Let me know if this helps resolve your
>>>>>>> questions,
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>          please feel
>>>>>>>>>>>>>>>>>          > >>>>>>> free to
>>>>>>>>>>>>>>>>>          > >>>>>>> add anything else on your mind.
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>          > >>>>>>> Adam
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
>>> Matthias J.
>>>>>>> Sax <
>>>>>>>>>>>>>>>>>          > >>>>>>> matthias@confluent.io <mailto:
>>>>>>>>>> matthias@confluent.io>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> wrote:
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>> Hi,
>>>>>>>>>>>>>>>>>          > >>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> I am just catching up on this thread. I
>>>>> did
>>>>>>> not
>>>>>>>>>> read
>>>>>>>>>>>>>>>>>          everything so
>>>>>>>>>>>>>>>>>          > >>>>>>>> far,
>>>>>>>>>>>>>>>>>          > >>>>>>>> but want to share couple of initial
>>>>> thoughts:
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Headers: I think there is a fundamental
>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>          between header
>>>>>>>>>>>>>>>>>          > >>>>>>>> usage
>>>>>>>>>>>>>>>>>          > >>>>>>>> in this KIP and KP-258. For 258, we add
>>>>>>> headers
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>          changelog topic
>>>>>>>>>>>>>>>>>          > >>>>>>>> that
>>>>>>>>>>>>>>>>>          > >>>>>>>> are owned by Kafka Streams and nobody
>>>>> else is
>>>>>>>>>> supposed
>>>>>>>>>>>>>>>>>          to write
>>>>>>>>>>>>>>>>>          > into
>>>>>>>>>>>>>>>>>          > >>>>>>>> them. In fact, no user header are
>>> written
>>>>> into
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          changelog topic
>>>>>>>>>>>>>>>>>          > >>>>>>>> and
>>>>>>>>>>>>>>>>>          > >>>>>>>> thus, there are not conflicts.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Nevertheless, I don't see a big issue
>>> with
>>>>>>> using
>>>>>>>>>>>>>>>>>          headers within
>>>>>>>>>>>>>>>>>          > >>>>>>>> Streams.
>>>>>>>>>>>>>>>>>          > >>>>>>>> As long as we document it, we can have
>>>>> some
>>>>>>>>>> "reserved"
>>>>>>>>>>>>>>>>>          header keys
>>>>>>>>>>>>>>>>>          > >>>>>>>> and
>>>>>>>>>>>>>>>>>          > >>>>>>>> users are not allowed to use when
>>>>> processing
>>>>>>>>>> data with
>>>>>>>>>>>>>>>>>          Kafka
>>>>>>>>>>>>>>>>>          > Streams.
>>>>>>>>>>>>>>>>>          > >>>>>>>> IMHO, this should be ok.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> I think there is a safe way to avoid
>>>>>>> conflicts,
>>>>>>>>>> since
>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>          > headers
>>>>>>>>>>>>>>>>>          > >>>>>>>> are
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>> only needed in internal topics (I
>>> think):
>>>>>>>>>>>>>>>>>          > >>>>>>>>> For internal and changelog topics, we
>>> can
>>>>>>>>>> namespace
>>>>>>>>>>>>>>>>>          all headers:
>>>>>>>>>>>>>>>>>          > >>>>>>>>> * user-defined headers are namespaced
>>> as
>>>>>>>>>> "external."
>>>>>>>>>>>>>>>>> +
>>>>>>>>>>>>>>>>>          headerKey
>>>>>>>>>>>>>>>>>          > >>>>>>>>> * internal headers are namespaced as
>>>>>>>>>> "internal." +
>>>>>>>>>>>>>>>>>          headerKey
>>>>>>>>>>>>>>>>>          > >>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>> While name spacing would be possible,
>>> it
>>>>>>> would
>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > >>>>>>>> deserialize
>>>>>>>>>>>>>>>>>          > >>>>>>>> user headers what implies a runtime
>>>>> overhead.
>>>>>>> I
>>>>>>>>>> would
>>>>>>>>>>>>>>>>>          suggest to
>>>>>>>>>>>>>>>>>          > no
>>>>>>>>>>>>>>>>>          > >>>>>>>> namespace for now to avoid the
>>> overhead.
>>>>> If
>>>>>>> this
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> becomes a
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > problem in
>>>>>>>>>>>>>>>>>          > >>>>>>>> the future, we can still add name
>>> spacing
>>>>>>> later
>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> My main concern about the design it the
>>>>> type
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          result KTable:
>>>>>>>>>>>>>>>>>          > >>>>>>>> If I
>>>>>>>>>>>>>>>>>          > >>>>>>>> understood the proposal correctly,
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> KTable<K1,V3> joinedTable =
>>>>>>>>>> table1.join(table2,...);
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> implies that the `joinedTable` has the
>>>>> same
>>>>>>> key
>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>          left input
>>>>>>>>>>>>>>>>>          > >>>>>>>> table.
>>>>>>>>>>>>>>>>>          > >>>>>>>> IMHO, this does not work because if
>>> table2
>>>>>>>>>> contains
>>>>>>>>>>>>>>>>>          multiple rows
>>>>>>>>>>>>>>>>>          > >>>>>>>> that
>>>>>>>>>>>>>>>>>          > >>>>>>>> join with a record in table1 (what is
>>> the
>>>>> main
>>>>>>>>>> purpose
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>          > foreign
>>>>>>>>>>>>>>>>>          > >>>>>>>> key
>>>>>>>>>>>>>>>>>          > >>>>>>>> join), the result table would only
>>>>> contain a
>>>>>>>>>> single
>>>>>>>>>>>>>>>>>          join result,
>>>>>>>>>>>>>>>>>          > but
>>>>>>>>>>>>>>>>>          > >>>>>>>> not
>>>>>>>>>>>>>>>>>          > >>>>>>>> multiple.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Example:
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> table1 input stream: <A,X>
>>>>>>>>>>>>>>>>>          > >>>>>>>> table2 input stream: <a,(A,1)>,
>>> <b,(A,2)>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> We use table2 value a foreign key to
>>>>> table1
>>>>>>> key
>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>          "A" joins).
>>>>>>>>>>>>>>>>>          > If
>>>>>>>>>>>>>>>>>          > >>>>>>>> the
>>>>>>>>>>>>>>>>>          > >>>>>>>> result key is the same key as key of
>>>>> table1,
>>>>>>> this
>>>>>>>>>>>>>>>>>          implies that the
>>>>>>>>>>>>>>>>>          > >>>>>>>> result can either be <A, join(X,1)> or
>>> <A,
>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>          but not
>>>>>>>>>>>>>>>>>          > both.
>>>>>>>>>>>>>>>>>          > >>>>>>>> Because the share the same key,
>>> whatever
>>>>>>> result
>>>>>>>>>> record
>>>>>>>>>>>>>>>>>          we emit
>>>>>>>>>>>>>>>>>          > later,
>>>>>>>>>>>>>>>>>          > >>>>>>>> overwrite the previous result.
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> This is the reason why Jan originally
>>>>> proposed
>>>>>>>>>> to use
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>          > combination
>>>>>>>>>>>>>>>>>          > >>>>>>>> of
>>>>>>>>>>>>>>>>>          > >>>>>>>> both primary keys of the input tables
>>> as
>>>>> key
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          output table.
>>>>>>>>>>>>>>>>>          > >>>>>>>> This
>>>>>>>>>>>>>>>>>          > >>>>>>>> makes the keys of the output table
>>> unique
>>>>> and
>>>>>>> we
>>>>>>>>>> can
>>>>>>>>>>>>>>>>>          store both in
>>>>>>>>>>>>>>>>>          > >>>>>>>> the
>>>>>>>>>>>>>>>>>          > >>>>>>>> output table:
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>>>>>>>>>> join(X,2)>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>>>>>>>>>>>          > >>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>> Just on remark here.
>>>>>>>>>>>>>>>>>          > >>>>>>>>> The high-watermark could be
>>> disregarded.
>>>>> The
>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>          about the
>>>>>>>>>>>>>>>>>          > >>>>>>>>> forward
>>>>>>>>>>>>>>>>>          > >>>>>>>>> depends on the size of the aggregated
>>>>> map.
>>>>>>>>>>>>>>>>>          > >>>>>>>>> Only 1 element long maps would be
>>>>> unpacked
>>>>>>> and
>>>>>>>>>>>>>>>>>          forwarded. 0
>>>>>>>>>>>>>>>>>          > element
>>>>>>>>>>>>>>>>>          > >>>>>>>>> maps
>>>>>>>>>>>>>>>>>          > >>>>>>>>> would be published as delete. Any
>>> other
>>>>> count
>>>>>>>>>>>>>>>>>          > >>>>>>>>> of map entries is in "waiting for
>>> correct
>>>>>>>>>> deletes to
>>>>>>>>>>>>>>>>>          > arrive"-state.
>>>>>>>>>>>>>>>>>          > >>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
>>>>> wrote:
>>>>>>>>>>>>>>>>>          > >>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>> It does look like I could replace the
>>>>> second
>>>>>>>>>>>>>>>>>          repartition store
>>>>>>>>>>>>>>>>>          > and
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> highwater store with a groupBy and
>>>>> reduce.
>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>          it looks
>>>>>>>>>>>>>>>>>          > like
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> I
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> would
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> still need to store the highwater
>>> value
>>>>>>> within
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          materialized
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> store,
>>>>>>>>>>>>>>>>>          > >>>>>>>>>>
>>>>>>>>>>>>>>>>>          > >>>>>>>>>> to
>>>>>>>>>>>>>>>>>          > >>>>>>>>> compare the arrival of out-of-order
>>>>> records
>>>>>>>>>> (assuming
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          > >>>>>>>>> understanding
>>>>>>>>>>>>>>>>>          > >>>>>>>>> of
>>>>>>>>>>>>>>>>>          > >>>>>>>>> THIS is correct...). This in effect is
>>>>> the
>>>>>>> same
>>>>>>>>>> as
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>          design I
>>>>>>>>>>>>>>>>>          > have
>>>>>>>>>>>>>>>>>          > >>>>>>>>> now,
>>>>>>>>>>>>>>>>>          > >>>>>>>>> just with the two tables merged
>>> together.
>>>>>>>>>>>>>>>>>          > >>>>>>>>>
>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Guozhang and Matthias

I know both of you are quite busy, but we've gotten this KIP to a point
where we need more guidance on the API (perhaps a bit of a tie-breaker, if
you will). If you have anyone else you may think should look at this,
please tag them accordingly.

The scenario is as such:

Current Option:
API:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-PublicInterfaces
1) Rekey the data to CombinedKey, and shuffles it to the partition with the
foreignKey (repartition 1)
2) Join the data
3) Shuffle the data back to the original node (repartition 2)
4) Resolve out-of-order arrival / race condition due to foreign-key changes.

Alternate Option:
Perform #1 and #2 above, and return a KScatteredTable.
- It would be keyed on a wrapped key function: <CombinedKey<KO, K>, VR> (KO
= Other Table Key, K = This Table Key, VR = Joined Result)
- KScatteredTable.resolve() would perform #3 and #4 but otherwise a user
would be able to perform additional functions directly from the
KScatteredTable (TBD - currently out of scope).
- John's analysis 2-emails up is accurate as to the tradeoffs.

Current Option is coded as-is. Alternate option is possible, but will
require for implementation details to be made in the API and some exposure
of new data structures into the API (ie: CombinedKey).

I appreciate any insight into this.

Thanks.

On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi John
>
> Thanks for your feedback and assistance. I think your summary is accurate
> from my perspective. Additionally, I would like to add that there is a risk
> of inconsistent final states without performing the resolution. This is a
> major concern for me as most of the data I have dealt with is produced by
> relational databases. We have seen a number of cases where a user in the
> Rails UI has modified the field (foreign key), realized they made a
> mistake, and then updated the field again with a new key. The events are
> propagated out as they are produced, and as such we have had real-world
> cases where these inconsistencies were propagated downstream as the final
> values due to the race conditions in the fanout of the data.
>
> This solution that I propose values correctness of the final result over
> other factors.
>
> We could always move this function over to using a KScatteredTable
> implementation in the future, and simply deprecate it this join API in
> time. I think I would like to hear more from some of the other major
> committers on which course of action they would think is best before any
> more coding is done.
>
> Thanks again
>
> Adam
>
>
> On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io> wrote:
>
>> Hi Jan and Adam,
>>
>> Wow, thanks for doing that test, Adam. Those results are encouraging.
>>
>> Thanks for your performance experience as well, Jan. I agree that avoiding
>> unnecessary join outputs is especially important when the fan-out is so
>> high. I suppose this could also be built into the implementation we're
>> discussing, but it wouldn't have to be specified in the KIP (since it's an
>> API-transparent optimization).
>>
>> As far as whether or not to re-repartition the data, I didn't bring it up
>> because it sounded like the two of you agreed to leave the KIP as-is,
>> despite the disagreement.
>>
>> If you want my opinion, I feel like both approaches are reasonable.
>> It sounds like Jan values more the potential for developers to optimize
>> their topologies to re-use the intermediate nodes, whereas Adam places
>> more
>> value on having a single operator that people can use without extra steps
>> at the end.
>>
>> Personally, although I do find it exceptionally annoying when a framework
>> gets in my way when I'm trying to optimize something, it seems better to
>> go
>> for a single operation.
>> * Encapsulating the internal transitions gives us significant latitude in
>> the implementation (for example, joining only at the end, not in the
>> middle
>> to avoid extra data copying and out-of-order resolution; how we represent
>> the first repartition keys (combined keys vs. value vectors), etc.). If we
>> publish something like a KScatteredTable with the right-partitioned joined
>> data, then the API pretty much locks in the implementation as well.
>> * The API seems simpler to understand and use. I do mean "seems"; if
>> anyone
>> wants to make the case that KScatteredTable is actually simpler, I think
>> hypothetical usage code would help. From a relational algebra perspective,
>> it seems like KTable.join(KTable) should produce a new KTable in all
>> cases.
>> * That said, there might still be room in the API for a different
>> operation
>> like what Jan has proposed to scatter a KTable, and then do things like
>> join, re-group, etc from there... I'm not sure; I haven't thought through
>> all the consequences yet.
>>
>> This is all just my opinion after thinking over the discussion so far...
>> -John
>>
>> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <ad...@gmail.com>
>> wrote:
>>
>> > Updated the PR to take into account John's feedback.
>> >
>> > I did some preliminary testing for the performance of the prefixScan. I
>> > have attached the file, but I will also include the text in the body
>> here
>> > for archival purposes (I am not sure what happens to attached files). I
>> > also updated the PR and the KIP accordingly.
>> >
>> > Summary: It scales exceptionally well for scanning large values of
>> > records. As Jan mentioned previously, the real issue would be more
>> around
>> > processing the resulting records after obtaining them. For instance, it
>> > takes approximately ~80-120 mS to flush the buffer and a further
>> ~35-85mS
>> > to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
>> > through the records just to generate a simple count takes ~ 40 times
>> longer
>> > than the flush + scan combined.
>> >
>> >
>> ============================================================================================
>> > Setup:
>> >
>> >
>> ============================================================================================
>> > Java 9 with default settings aside from a 512 MB heap (Xmx512m, Xms512m)
>> > CPU: i7 2.2 Ghz.
>> >
>> > Note: I am using a slightly-modified, directly-accessible Kafka Streams
>> > RocksDB
>> > implementation (RocksDB.java, basically just avoiding the
>> > ProcessorContext).
>> > There are no modifications to the default RocksDB values provided in the
>> > 2.1/trunk release.
>> >
>> >
>> > keysize = 128 bytes
>> > valsize = 512 bytes
>> >
>> > Step 1:
>> > Write X positive matching events: (key = prefix + left-padded
>> > auto-incrementing integer)
>> > Step 2:
>> > Write 10X negative matching events (key = left-padded auto-incrementing
>> > integer)
>> > Step 3:
>> > Perform flush
>> > Step 4:
>> > Perform prefixScan
>> > Step 5:
>> > Iterate through return Iterator and validate the count of expected
>> events.
>> >
>> >
>> >
>> ============================================================================================
>> > Results:
>> >
>> >
>> ============================================================================================
>> > X = 1k (11k events total)
>> > Flush Time = 39 mS
>> > Scan Time = 7 mS
>> > 6.9 MB disk
>> >
>> >
>> --------------------------------------------------------------------------------------------
>> > X = 10k (110k events total)
>> > Flush Time = 45 mS
>> > Scan Time = 8 mS
>> > 127 MB
>> >
>> >
>> --------------------------------------------------------------------------------------------
>> > X = 100k (1.1M events total)
>> > Test1:
>> > Flush Time = 60 mS
>> > Scan Time = 12 mS
>> > 678 MB
>> >
>> > Test2:
>> > Flush Time = 45 mS
>> > Scan Time = 7 mS
>> > 576 MB
>> >
>> >
>> --------------------------------------------------------------------------------------------
>> > X = 1MB (11M events total)
>> > Test1:
>> > Flush Time = 52 mS
>> > Scan Time = 19 mS
>> > 7.2 GB
>> >
>> > Test2:
>> > Flush Time = 84 mS
>> > Scan Time = 34 mS
>> > 9.1 GB
>> >
>> >
>> --------------------------------------------------------------------------------------------
>> > X = 2.5M (27.5M events total)
>> > Test1:
>> > Flush Time = 82 mS
>> > Scan Time = 63 mS
>> > 17GB - 276 sst files
>> >
>> > Test2:
>> > Flush Time = 116 mS
>> > Scan Time = 35 mS
>> > 23GB - 361 sst files
>> >
>> > Test3:
>> > Flush Time = 103 mS
>> > Scan Time = 82 mS
>> > 19 GB - 300 sst files
>> >
>> >
>> --------------------------------------------------------------------------------------------
>> >
>> > I had to limit my testing on my laptop to X = 2.5M events. I tried to go
>> > to X = 10M (110M events) but RocksDB was going into the 100GB+ range
>> and my
>> > laptop ran out of disk. More extensive testing could be done but I
>> suspect
>> > that it would be in line with what we're seeing in the results above.
>> >
>> >
>> >
>> >
>> >
>> >
>> > At this point in time, I think the only major discussion point is really
>> > around what Jan and I have disagreed on: repartitioning back + resolving
>> > potential out of order issues or leaving that up to the client to
>> handle.
>> >
>> >
>> > Thanks folks,
>> >
>> > Adam
>> >
>> >
>> > On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Ja...@trivago.com>
>> > wrote:
>> >
>> >>
>> >>
>> >> On 29.11.2018 15:14, John Roesler wrote:
>> >> > Hi all,
>> >> >
>> >> > Sorry that this discussion petered out... I think the 2.1 release
>> >> caused an
>> >> > extended distraction that pushed it off everyone's radar (which was
>> >> > precisely Adam's concern). Personally, I've also had some extend
>> >> > distractions of my own that kept (and continue to keep) me
>> preoccupied.
>> >> >
>> >> > However, calling for a vote did wake me up, so I guess Jan was on the
>> >> right
>> >> > track!
>> >> >
>> >> > I've gone back and reviewed the whole KIP document and the prior
>> >> > discussion, and I'd like to offer a few thoughts:
>> >> >
>> >> > API Thoughts:
>> >> >
>> >> > 1. If I read the KIP right, you are proposing a many-to-one join.
>> Could
>> >> we
>> >> > consider naming it manyToOneJoin? Or, if you prefer, flip the design
>> >> around
>> >> > and make it a oneToManyJoin?
>> >> >
>> >> > The proposed name "joinOnForeignKey" disguises the join type, and it
>> >> seems
>> >> > like it might trick some people into using it for a one-to-one join.
>> >> This
>> >> > would work, of course, but it would be super inefficient compared to
>> a
>> >> > simple rekey-and-join.
>> >> >
>> >> > 2. I might have missed it, but I don't think it's specified whether
>> >> it's an
>> >> > inner, outer, or left join. I'm guessing an outer join, as
>> (neglecting
>> >> IQ),
>> >> > the rest can be achieved by filtering or by handling it in the
>> >> ValueJoiner.
>> >> >
>> >> > 3. The arg list to joinOnForeignKey doesn't look quite right.
>> >> > 3a. Regarding Serialized: There are a few different paradigms in
>> play in
>> >> > the Streams API, so it's confusing, but instead of three Serialized
>> >> args, I
>> >> > think it would be better to have one that allows (optionally) setting
>> >> the 4
>> >> > incoming serdes. The result serde is defined by the Materialized. The
>> >> > incoming serdes can be optional because they might already be
>> available
>> >> on
>> >> > the source KTables, or the default serdes from the config might be
>> >> > applicable.
>> >> >
>> >> > 3b. Is the StreamPartitioner necessary? The other joins don't allow
>> >> setting
>> >> > one, and it seems like it might actually be harmful, since the rekey
>> >> > operation needs to produce results that are co-partitioned with the
>> >> "other"
>> >> > KTable.
>> >> >
>> >> > 4. I'm fine with the "reserved word" header, but I didn't actually
>> >> follow
>> >> > what Matthias meant about namespacing requiring "deserializing" the
>> >> record
>> >> > header. The headers are already Strings, so I don't think that
>> >> > deserialization is required. If we applied the namespace at source
>> nodes
>> >> > and stripped it at sink nodes, this would be practically no overhead.
>> >> The
>> >> > advantage of the namespace idea is that no public API change wrt
>> headers
>> >> > needs to happen, and no restrictions need to be placed on users'
>> >> headers.
>> >> >
>> >> > (Although I'm wondering if we can get away without the header at
>> all...
>> >> > stay tuned)
>> >> >
>> >> > 5. I also didn't follow the discussion about the HWM table growing
>> >> without
>> >> > bound. As I read it, the HWM table is effectively implementing OCC to
>> >> > resolve the problem you noted with disordering when the rekey is
>> >> > reversed... particularly notable when the FK changes. As such, it
>> only
>> >> > needs to track the most recent "version" (the offset in the source
>> >> > partition) of each key. Therefore, it should have the same number of
>> >> keys
>> >> > as the source table at all times.
>> >> >
>> >> > I see that you are aware of KIP-258, which I think might be relevant
>> in
>> >> a
>> >> > couple of ways. One: it's just about storing the timestamp in the
>> state
>> >> > store, but the ultimate idea is to effectively use the timestamp as
>> an
>> >> OCC
>> >> > "version" to drop disordered updates. You wouldn't want to use the
>> >> > timestamp for this operation, but if you were to use a similar
>> >> mechanism to
>> >> > store the source offset in the store alongside the re-keyed values,
>> then
>> >> > you could avoid a separate table.
>> >> >
>> >> > 6. You and Jan have been thinking about this for a long time, so I've
>> >> > probably missed something here, but I'm wondering if we can avoid the
>> >> HWM
>> >> > tracking at all and resolve out-of-order during a final join
>> instead...
>> >> >
>> >> > Let's say we're joining a left table (Integer K: Letter FK, (other
>> >> data))
>> >> > to a right table (Letter K: (some data)).
>> >> >
>> >> > Left table:
>> >> > 1: (A, xyz)
>> >> > 2: (B, asd)
>> >> >
>> >> > Right table:
>> >> > A: EntityA
>> >> > B: EntityB
>> >> >
>> >> > We could do a rekey as you proposed with a combined key, but not
>> >> > propagating the value at all..
>> >> > Rekey table:
>> >> > A-1: (dummy value)
>> >> > B-2: (dummy value)
>> >> >
>> >> > Which we then join with the right table to produce:
>> >> > A-1: EntityA
>> >> > B-2: EntityB
>> >> >
>> >> > Which gets rekeyed back:
>> >> > 1: A, EntityA
>> >> > 2: B, EntityB
>> >> >
>> >> > And finally we do the actual join:
>> >> > Result table:
>> >> > 1: ((A, xyz), EntityA)
>> >> > 2: ((B, asd), EntityB)
>> >> >
>> >> > The thing is that in that last join, we have the opportunity to
>> compare
>> >> the
>> >> > current FK in the left table with the incoming PK of the right
>> table. If
>> >> > they don't match, we just drop the event, since it must be outdated.
>> >> >
>> >>
>> >> > In your KIP, you gave an example in which (1: A, xyz) gets updated to
>> >> (1:
>> >> > B, xyz), ultimately yielding a conundrum about whether the final
>> state
>> >> > should be (1: null) or (1: joined-on-B). With the algorithm above,
>> you
>> >> > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
>> >> > EntityB)). It seems like this does give you enough information to
>> make
>> >> the
>> >> > right choice, regardless of disordering.
>> >>
>> >> Will check Adams patch, but this should work. As mentioned often I am
>> >> not convinced on partitioning back for the user automatically. I think
>> >> this is the real performance eater ;)
>> >>
>> >> >
>> >> >
>> >> > 7. Last thought... I'm a little concerned about the performance of
>> the
>> >> > range scans when records change in the right table. You've said that
>> >> you've
>> >> > been using the algorithm you presented in production for a while. Can
>> >> you
>> >> > give us a sense of the performance characteristics you've observed?
>> >> >
>> >>
>> >> Make it work, make it fast, make it beautiful. The topmost thing here
>> is
>> >> / was correctness. In practice I do not measure the performance of the
>> >> range scan. Usual cases I run this with is emitting 500k - 1kk rows
>> >> on a left hand side change. The range scan is just the work you gotta
>> >> do, also when you pack your data into different formats, usually the
>> >> rocks performance is very tight to the size of the data and we can't
>> >> really change that. It is more important for users to prevent useless
>> >> updates to begin with. My left hand side is guarded to drop changes
>> that
>> >> are not going to change my join output.
>> >>
>> >> usually it's:
>> >>
>> >> drop unused fields and then don't forward if old.equals(new)
>> >>
>> >> regarding to the performance of creating an iterator for smaller
>> >> fanouts, users can still just do a group by first then anyways.
>> >>
>> >>
>> >>
>> >> > I could only think of one alternative, but I'm not sure if it's
>> better
>> >> or
>> >> > worse... If the first re-key only needs to preserve the original key,
>> >> as I
>> >> > proposed in #6, then we could store a vector of keys in the value:
>> >> >
>> >> > Left table:
>> >> > 1: A,...
>> >> > 2: B,...
>> >> > 3: A,...
>> >> >
>> >> > Gets re-keyed:
>> >> > A: [1, 3]
>> >> > B: [2]
>> >> >
>> >> > Then, the rhs part of the join would only need a regular single-key
>> >> lookup.
>> >> > Of course we have to deal with the problem of large values, as
>> there's
>> >> no
>> >> > bound on the number of lhs records that can reference rhs records.
>> >> Offhand,
>> >> > I'd say we could page the values, so when one row is past the
>> >> threshold, we
>> >> > append the key for the next page. Then in most cases, it would be a
>> >> single
>> >> > key lookup, but for large fan-out updates, it would be one per (max
>> >> value
>> >> > size)/(avg lhs key size).
>> >> >
>> >> > This seems more complex, though... Plus, I think there's some extra
>> >> > tracking we'd need to do to know when to emit a retraction. For
>> example,
>> >> > when record 1 is deleted, the re-key table would just have (A: [3]).
>> >> Some
>> >> > kind of tombstone is needed so that the join result for 1 can also be
>> >> > retracted.
>> >> >
>> >> > That's all!
>> >> >
>> >> > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
>> >> > discussion has been slow.
>> >> > -John
>> >> >
>> >> >
>> >> > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
>> Jan.Filipiak@trivago.com>
>> >> > wrote:
>> >> >
>> >> >> Id say you can just call the vote.
>> >> >>
>> >> >> that happens all the time, and if something comes up, it just goes
>> back
>> >> >> to discuss.
>> >> >>
>> >> >> would not expect to much attention with another another email in
>> this
>> >> >> thread.
>> >> >>
>> >> >> best Jan
>> >> >>
>> >> >> On 09.10.2018 13:56, Adam Bellemare wrote:
>> >> >>> Hello Contributors
>> >> >>>
>> >> >>> I know that 2.1 is about to be released, but I do need to bump
>> this to
>> >> >> keep
>> >> >>> visibility up. I am still intending to push this through once
>> >> contributor
>> >> >>> feedback is given.
>> >> >>>
>> >> >>> Main points that need addressing:
>> >> >>> 1) Any way (or benefit) in structuring the current singular graph
>> node
>> >> >> into
>> >> >>> multiple nodes? It has a whopping 25 parameters right now. I am a
>> bit
>> >> >> fuzzy
>> >> >>> on how the optimizations are supposed to work, so I would
>> appreciate
>> >> any
>> >> >>> help on this aspect.
>> >> >>>
>> >> >>> 2) Overall strategy for joining + resolving. This thread has much
>> >> >> discourse
>> >> >>> between Jan and I between the current highwater mark proposal and a
>> >> >> groupBy
>> >> >>> + reduce proposal. I am of the opinion that we need to strictly
>> handle
>> >> >> any
>> >> >>> chance of out-of-order data and leave none of it up to the
>> consumer.
>> >> Any
>> >> >>> comments or suggestions here would also help.
>> >> >>>
>> >> >>> 3) Anything else that you see that would prevent this from moving
>> to a
>> >> >> vote?
>> >> >>>
>> >> >>> Thanks
>> >> >>>
>> >> >>> Adam
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> >> >> adam.bellemare@gmail.com>
>> >> >>> wrote:
>> >> >>>
>> >> >>>> Hi Jan
>> >> >>>>
>> >> >>>> With the Stores.windowStoreBuilder and
>> Stores.persistentWindowStore,
>> >> you
>> >> >>>> actually only need to specify the amount of segments you want and
>> how
>> >> >> large
>> >> >>>> they are. To the best of my understanding, what happens is that
>> the
>> >> >>>> segments are automatically rolled over as new data with new
>> >> timestamps
>> >> >> are
>> >> >>>> created. We use this exact functionality in some of the work done
>> >> >>>> internally at my company. For reference, this is the hopping
>> windowed
>> >> >> store.
>> >> >>>>
>> >> >>>>
>> >> >>
>> >>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>> >> >>>>
>> >> >>>> In the code that I have provided, there are going to be two 24h
>> >> >> segments.
>> >> >>>> When a record is put into the windowStore, it will be inserted at
>> >> time
>> >> >> T in
>> >> >>>> both segments. The two segments will always overlap by 12h. As
>> time
>> >> >> goes on
>> >> >>>> and new records are added (say at time T+12h+), the oldest segment
>> >> will
>> >> >> be
>> >> >>>> automatically deleted and a new segment created. The records are
>> by
>> >> >> default
>> >> >>>> inserted with the context.timestamp(), such that it is the record
>> >> time,
>> >> >> not
>> >> >>>> the clock time, which is used.
>> >> >>>>
>> >> >>>> To the best of my understanding, the timestamps are retained when
>> >> >>>> restoring from the changelog.
>> >> >>>>
>> >> >>>> Basically, this is heavy-handed way to deal with TTL at a
>> >> segment-level,
>> >> >>>> instead of at an individual record level.
>> >> >>>>
>> >> >>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>> >> Jan.Filipiak@trivago.com>
>> >> >>>> wrote:
>> >> >>>>
>> >> >>>>> Will that work? I expected it to blow up with ClassCastException
>> or
>> >> >>>>> similar.
>> >> >>>>>
>> >> >>>>> You either would have to specify the window you fetch/put or
>> iterate
>> >> >>>>> across all windows the key was found in right?
>> >> >>>>>
>> >> >>>>> I just hope the window-store doesn't check stream-time under the
>> >> hoods
>> >> >>>>> that would be a questionable interface.
>> >> >>>>>
>> >> >>>>> If it does: did you see my comment on checking all the windows
>> >> earlier?
>> >> >>>>> that would be needed to actually give reasonable time gurantees.
>> >> >>>>>
>> >> >>>>> Best
>> >> >>>>>
>> >> >>>>>
>> >> >>>>>
>> >> >>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> >> >>>>>> Hi Jan
>> >> >>>>>>
>> >> >>>>>> Check for  " highwaterMat " in the PR. I only changed the state
>> >> store,
>> >> >>>>> not
>> >> >>>>>> the ProcessorSupplier.
>> >> >>>>>>
>> >> >>>>>> Thanks,
>> >> >>>>>> Adam
>> >> >>>>>>
>> >> >>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> >> >> Jan.Filipiak@trivago.com
>> >> >>>>>>
>> >> >>>>>> wrote:
>> >> >>>>>>
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>> >> >>>>>>>
>> >> >>>>>>>> @Guozhang
>> >> >>>>>>>>
>> >> >>>>>>>> Thanks for the information. This is indeed something that
>> will be
>> >> >>>>>>>> extremely
>> >> >>>>>>>> useful for this KIP.
>> >> >>>>>>>>
>> >> >>>>>>>> @Jan
>> >> >>>>>>>> Thanks for your explanations. That being said, I will not be
>> >> moving
>> >> >>>>> ahead
>> >> >>>>>>>> with an implementation using reshuffle/groupBy solution as you
>> >> >>>>> propose.
>> >> >>>>>>>> That being said, if you wish to implement it yourself off of
>> my
>> >> >>>>> current PR
>> >> >>>>>>>> and submit it as a competitive alternative, I would be more
>> than
>> >> >>>>> happy to
>> >> >>>>>>>> help vet that as an alternate solution. As it stands right
>> now,
>> >> I do
>> >> >>>>> not
>> >> >>>>>>>> really have more time to invest into alternatives without
>> there
>> >> >> being
>> >> >>>>> a
>> >> >>>>>>>> strong indication from the binding voters which they would
>> >> prefer.
>> >> >>>>>>>>
>> >> >>>>>>>>
>> >> >>>>>>> Hey, total no worries. I think I personally gave up on the
>> streams
>> >> >> DSL
>> >> >>>>> for
>> >> >>>>>>> some time already, otherwise I would have pulled this KIP
>> through
>> >> >>>>> already.
>> >> >>>>>>> I am currently reimplementing my own DSL based on PAPI.
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>>> I will look at finishing up my PR with the windowed state
>> store
>> >> in
>> >> >> the
>> >> >>>>>>>> next
>> >> >>>>>>>> week or so, exercising it via tests, and then I will come back
>> >> for
>> >> >>>>> final
>> >> >>>>>>>> discussions. In the meantime, I hope that any of the binding
>> >> voters
>> >> >>>>> could
>> >> >>>>>>>> take a look at the KIP in the wiki. I have updated it
>> according
>> >> to
>> >> >> the
>> >> >>>>>>>> latest plan:
>> >> >>>>>>>>
>> >> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> >> >>>>>>>> Support+non-key+joining+in+KTable
>> >> >>>>>>>>
>> >> >>>>>>>> I have also updated the KIP PR to use a windowed store. This
>> >> could
>> >> >> be
>> >> >>>>>>>> replaced by the results of KIP-258 whenever they are
>> completed.
>> >> >>>>>>>> https://github.com/apache/kafka/pull/5527
>> >> >>>>>>>>
>> >> >>>>>>>> Thanks,
>> >> >>>>>>>>
>> >> >>>>>>>> Adam
>> >> >>>>>>>>
>> >> >>>>>>>
>> >> >>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated
>> in
>> >> the
>> >> >>>>> PR?
>> >> >>>>>>> expected it to change to Windowed<K>,Long Missing something?
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>>>
>> >> >>>>>>>>
>> >> >>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
>> >> wangguoz@gmail.com>
>> >> >>>>>>>> wrote:
>> >> >>>>>>>>
>> >> >>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link,
>> >> as it
>> >> >>>>> is
>> >> >>>>>>>>> for
>> >> >>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258
>> we do
>> >> >>>>> want to
>> >> >>>>>>>>> have "handling out-of-order data for source KTable" such that
>> >> >>>>> instead of
>> >> >>>>>>>>> blindly apply the updates to the materialized store, i.e.
>> >> following
>> >> >>>>>>>>> offset
>> >> >>>>>>>>> ordering, we will reject updates that are older than the
>> current
>> >> >>>>> key's
>> >> >>>>>>>>> timestamps, i.e. following timestamp ordering.
>> >> >>>>>>>>>
>> >> >>>>>>>>>
>> >> >>>>>>>>> Guozhang
>> >> >>>>>>>>>
>> >> >>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>> >> >> wangguoz@gmail.com>
>> >> >>>>>>>>> wrote:
>> >> >>>>>>>>>
>> >> >>>>>>>>> Hello Adam,
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
>> the
>> >> >> high
>> >> >>>>>>>>>> watermark store, now altered to be replaced with a window
>> >> store),
>> >> >> I
>> >> >>>>>>>>>> think
>> >> >>>>>>>>>> another current on-going KIP may actually help:
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >> >>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> This is for adding the timestamp into a key-value store
>> (i.e.
>> >> only
>> >> >>>>> for
>> >> >>>>>>>>>> non-windowed KTable), and then one of its usage, as
>> described
>> >> in
>> >> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that
>> we
>> >> can
>> >> >>>>> then
>> >> >>>>>>>>>> "reject" updates from the source topics if its timestamp is
>> >> >> smaller
>> >> >>>>> than
>> >> >>>>>>>>>> the current key's latest update timestamp. I think it is
>> very
>> >> >>>>> similar to
>> >> >>>>>>>>>> what you have in mind for high watermark based filtering,
>> while
>> >> >> you
>> >> >>>>> only
>> >> >>>>>>>>>> need to make sure that the timestamps of the joining records
>> >> are
>> >> >>>>>>>>>>
>> >> >>>>>>>>> correctly
>> >> >>>>>>>>>
>> >> >>>>>>>>>> inherited though the whole topology to the final stage.
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> Note that this KIP is for key-value store and hence
>> >> non-windowed
>> >> >>>>> KTables
>> >> >>>>>>>>>> only, but for windowed KTables we do not really have a good
>> >> >> support
>> >> >>>>> for
>> >> >>>>>>>>>> their joins anyways (
>> >> >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> >> >>>>>>>>>> I
>> >> >>>>>>>>>> think we can just consider non-windowed KTable-KTable
>> non-key
>> >> >> joins
>> >> >>>>> for
>> >> >>>>>>>>>> now. In which case, KIP-258 should help.
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> Guozhang
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>> >> >>>>> Jan.Filipiak@trivago.com
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> wrote:
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>> Hi Guozhang
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> Current highwater mark implementation would grow endlessly
>> >> based
>> >> >>>>> on
>> >> >>>>>>>>>>>> primary key of original event. It is a pair of (<this
>> table
>> >> >>>>> primary
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> key>,
>> >> >>>>>>>>>
>> >> >>>>>>>>>> <highest offset seen for that key>). This is used to
>> >> differentiate
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> between
>> >> >>>>>>>>>
>> >> >>>>>>>>>> late arrivals and new updates. My newest proposal would be
>> to
>> >> >>>>> replace
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> it
>> >> >>>>>>>>>
>> >> >>>>>>>>>> with a Windowed state store of Duration N. This would allow
>> the
>> >> >> same
>> >> >>>>>>>>>>>> behaviour, but cap the size based on time. This should
>> allow
>> >> for
>> >> >>>>> all
>> >> >>>>>>>>>>>> late-arriving events to be processed, and should be
>> >> customizable
>> >> >>>>> by
>> >> >>>>>>>>>>>> the
>> >> >>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
>> >> minutes
>> >> >> of
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> window,
>> >> >>>>>>>>>
>> >> >>>>>>>>>> or perhaps 7 days...).
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> Hi Adam, using time based retention can do the trick here.
>> >> Even
>> >> >>>>> if I
>> >> >>>>>>>>>>> would still like to see the automatic repartitioning
>> optional
>> >> >>>>> since I
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> would
>> >> >>>>>>>>>
>> >> >>>>>>>>>> just reshuffle again. With windowed store I am a little bit
>> >> >>>>> sceptical
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> about
>> >> >>>>>>>>>
>> >> >>>>>>>>>> how to determine the window. So esentially one could run
>> into
>> >> >>>>> problems
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> when
>> >> >>>>>>>>>
>> >> >>>>>>>>>> the rapid change happens near a window border. I will check
>> you
>> >> >>>>>>>>>>> implementation in detail, if its problematic, we could
>> still
>> >> >> check
>> >> >>>>>>>>>>> _all_
>> >> >>>>>>>>>>> windows on read with not to bad performance impact I guess.
>> >> Will
>> >> >>>>> let
>> >> >>>>>>>>>>> you
>> >> >>>>>>>>>>> know if the implementation would be correct as is. I
>> wouldn't
>> >> not
>> >> >>>>> like
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> to
>> >> >>>>>>>>>
>> >> >>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>> >> >> timestamp(B).
>> >> >>>>> I
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> think
>> >> >>>>>>>>>
>> >> >>>>>>>>>> we can't expect that.
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> @Jan
>> >> >>>>>>>>>>>> I believe I understand what you mean now - thanks for the
>> >> >>>>> diagram, it
>> >> >>>>>>>>>>>> did really help. You are correct that I do not have the
>> >> original
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> primary
>> >> >>>>>>>>>
>> >> >>>>>>>>>> key available, and I can see that if it was available then
>> you
>> >> >>>>> would be
>> >> >>>>>>>>>>>> able to add and remove events from the Map. That being
>> said,
>> >> I
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> encourage
>> >> >>>>>>>>>
>> >> >>>>>>>>>> you to finish your diagrams / charts just for clarity for
>> >> everyone
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> else.
>> >> >>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But
>> I
>> >> >>>>> understand
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> the benefits for the rest. Sorry about the original primary
>> >> key,
>> >> >> We
>> >> >>>>>>>>>>> have
>> >> >>>>>>>>>>> join and Group by implemented our own in PAPI and basically
>> >> not
>> >> >>>>> using
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> any
>> >> >>>>>>>>>
>> >> >>>>>>>>>> DSL (Just the abstraction). Completely missed that in
>> original
>> >> DSL
>> >> >>>>> its
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> not
>> >> >>>>>>>>>
>> >> >>>>>>>>>> there and just assumed it. total brain mess up on my end.
>> Will
>> >> >>>>> finish
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> the
>> >> >>>>>>>>>
>> >> >>>>>>>>>> chart as soon as i get a quite evening this week.
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>> My follow up question for you is, won't the Map stay inside
>> >> the
>> >> >>>>> State
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>>> Store indefinitely after all of the changes have
>> propagated?
>> >> >> Isn't
>> >> >>>>>>>>>>>> this
>> >> >>>>>>>>>>>> effectively the same as a highwater mark state store?
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
>> >> return
>> >> >>>>> `null`
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> and
>> >> >>>>>>>>>
>> >> >>>>>>>>>> the key is removed from the keyspace. But there is going to
>> be
>> >> a
>> >> >>>>> store
>> >> >>>>>>>>>>> 100%, the good thing is that I can use this store directly
>> for
>> >> >>>>>>>>>>> materialize() / enableSendingOldValues() is a regular
>> store,
>> >> >>>>> satisfying
>> >> >>>>>>>>>>> all gurantees needed for further groupby / join. The
>> Windowed
>> >> >>>>> store is
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> not
>> >> >>>>>>>>>
>> >> >>>>>>>>>> keeping the values, so for the next statefull operation we
>> >> would
>> >> >>>>>>>>>>> need to instantiate an extra store. or we have the window
>> >> store
>> >> >>>>> also
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> have
>> >> >>>>>>>>>
>> >> >>>>>>>>>> the values then.
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>> Long story short. if we can flip in a custom group by
>> before
>> >> >>>>>>>>>>> repartitioning to the original primary key i think it would
>> >> help
>> >> >>>>> the
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> users
>> >> >>>>>>>>>
>> >> >>>>>>>>>> big time in building efficient apps. Given the original
>> primary
>> >> >> key
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> issue I
>> >> >>>>>>>>>
>> >> >>>>>>>>>> understand that we do not have a solid foundation to build
>> on.
>> >> >>>>>>>>>>> Leaving primary key carry along to the user. very
>> >> unfortunate. I
>> >> >>>>> could
>> >> >>>>>>>>>>> understand the decision goes like that. I do not think its
>> a
>> >> good
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>> decision.
>> >> >>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> Thanks
>> >> >>>>>>>>>>>> Adam
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>> >> >>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>> >> dumbreprajakta311@gmail.com
>> >> >>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> wrote:
>> >> >>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>>>         please remove me from this group
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> >> Jan.Filipiak@trivago.com
>> >> >>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         wrote:
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > Hi Adam,
>> >> >>>>>>>>>>>>         >
>> >> >>>>>>>>>>>>         > give me some time, will make such a chart. last
>> >> time i
>> >> >>>>> didn't
>> >> >>>>>>>>>>>>         get along
>> >> >>>>>>>>>>>>         > well with giphy and ruined all your charts.
>> >> >>>>>>>>>>>>         > Hopefully i can get it done today
>> >> >>>>>>>>>>>>         >
>> >> >>>>>>>>>>>>         > On 08.09.2018 16:00, Adam Bellemare wrote:
>> >> >>>>>>>>>>>>         > > Hi Jan
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > > I have included a diagram of what I attempted
>> on
>> >> the
>> >> >>>>> KIP.
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         >
>> >> >>>>>>>>>>>>
>> >> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> >> >>>>>>>>>>>>
>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> >> >>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> >> >>>>>>>>>>>>         <
>> >> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> >> >>>>>>>>>>>>
>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> >> >>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > > I attempted this back at the start of my own
>> >> >>>>> implementation
>> >> >>>>>>>>>>>> of
>> >> >>>>>>>>>>>>         this
>> >> >>>>>>>>>>>>         > > solution, and since I could not get it to
>> work I
>> >> have
>> >> >>>>> since
>> >> >>>>>>>>>>>>         discarded the
>> >> >>>>>>>>>>>>         > > code. At this point in time, if you wish to
>> >> continue
>> >> >>>>> pursuing
>> >> >>>>>>>>>>>>         for your
>> >> >>>>>>>>>>>>         > > groupBy solution, I ask that you please
>> create a
>> >> >>>>> diagram on
>> >> >>>>>>>>>>>>         the KIP
>> >> >>>>>>>>>>>>         > > carefully explaining your solution. Please
>> feel
>> >> free
>> >> >> to
>> >> >>>>> use
>> >> >>>>>>>>>>>>         the image I
>> >> >>>>>>>>>>>>         > > just posted as a starting point. I am having
>> >> trouble
>> >> >>>>>>>>>>>>         understanding your
>> >> >>>>>>>>>>>>         > > explanations but I think that a carefully
>> >> constructed
>> >> >>>>> diagram
>> >> >>>>>>>>>>>>         will clear
>> >> >>>>>>>>>>>>         > up
>> >> >>>>>>>>>>>>         > > any misunderstandings. Alternately, please
>> post a
>> >> >>>>>>>>>>>>         comprehensive PR with
>> >> >>>>>>>>>>>>         > > your solution. I can only guess at what you
>> >> mean, and
>> >> >>>>> since I
>> >> >>>>>>>>>>>>         value my
>> >> >>>>>>>>>>>>         > own
>> >> >>>>>>>>>>>>         > > time as much as you value yours, I believe it
>> is
>> >> your
>> >> >>>>>>>>>>>>         responsibility to
>> >> >>>>>>>>>>>>         > > provide an implementation instead of me
>> trying to
>> >> >> guess.
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > > Adam
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> >> Jan.Filipiak@trivago.com
>> >> >>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > > wrote:
>> >> >>>>>>>>>>>>         > >
>> >> >>>>>>>>>>>>         > >> Hi James,
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >> nice to see you beeing interested. kafka
>> >> streams at
>> >> >>>>> this
>> >> >>>>>>>>>>>>         point supports
>> >> >>>>>>>>>>>>         > >> all sorts of joins as long as both streams
>> have
>> >> the
>> >> >>>>> same
>> >> >>>>>>>>>>>> key.
>> >> >>>>>>>>>>>>         > >> Adam is currently implementing a join where a
>> >> KTable
>> >> >>>>> and a
>> >> >>>>>>>>>>>>         KTable can
>> >> >>>>>>>>>>>>         > have
>> >> >>>>>>>>>>>>         > >> a one to many relation ship (1:n). We exploit
>> >> that
>> >> >>>>> rocksdb
>> >> >>>>>>>>>>>> is
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> a
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > >> datastore that keeps data sorted (At least
>> >> exposes an
>> >> >>>>> API to
>> >> >>>>>>>>>>>>         access the
>> >> >>>>>>>>>>>>         > >> stored data in a sorted fashion).
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >> I think the technical caveats are well
>> >> understood
>> >> >> now
>> >> >>>>> and we
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> are
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > basically
>> >> >>>>>>>>>>>>         > >> down to philosophy and API Design ( when Adam
>> >> sees
>> >> >> my
>> >> >>>>> newest
>> >> >>>>>>>>>>>>         message).
>> >> >>>>>>>>>>>>         > >> I have a lengthy track record of loosing
>> those
>> >> kinda
>> >> >>>>>>>>>>>>         arguments within
>> >> >>>>>>>>>>>>         > the
>> >> >>>>>>>>>>>>         > >> streams community and I have no clue why. So
>> I
>> >> >>>>> literally
>> >> >>>>>>>>>>>>         can't wait for
>> >> >>>>>>>>>>>>         > you
>> >> >>>>>>>>>>>>         > >> to churn through this thread and give you
>> >> opinion on
>> >> >>>>> how we
>> >> >>>>>>>>>>>>         should
>> >> >>>>>>>>>>>>         > design
>> >> >>>>>>>>>>>>         > >> the return type of the oneToManyJoin and how
>> >> many
>> >> >>>>> power we
>> >> >>>>>>>>>>>>         want to give
>> >> >>>>>>>>>>>>         > to
>> >> >>>>>>>>>>>>         > >> the user vs "simplicity" (where simplicity
>> isn't
>> >> >>>>> really that
>> >> >>>>>>>>>>>>         as users
>> >> >>>>>>>>>>>>         > still
>> >> >>>>>>>>>>>>         > >> need to understand it I argue)
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >> waiting for you to join in on the discussion
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >> Best Jan
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >> On 07.09.2018 15:49, James Kwan wrote:
>> >> >>>>>>>>>>>>         > >>
>> >> >>>>>>>>>>>>         > >>> I am new to this group and I found this
>> subject
>> >> >>>>>>>>>>>>         interesting.  Sounds
>> >> >>>>>>>>>>>>         > like
>> >> >>>>>>>>>>>>         > >>> you guys want to implement a join table of
>> two
>> >> >>>>> streams? Is
>> >> >>>>>>>>>>>> there
>> >> >>>>>>>>>>>>         > somewhere
>> >> >>>>>>>>>>>>         > >>> I can see the original requirement or
>> proposal?
>> >> >>>>>>>>>>>>         > >>>
>> >> >>>>>>>>>>>>         > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> >> Jan.Filipiak@trivago.com
>> >> >>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>> wrote:
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>> I'm currently testing using a Windowed
>> Store
>> >> to
>> >> >>>>> store the
>> >> >>>>>>>>>>>>         highwater
>> >> >>>>>>>>>>>>         > >>>>> mark.
>> >> >>>>>>>>>>>>         > >>>>> By all indications this should work fine,
>> >> with
>> >> >> the
>> >> >>>>> caveat
>> >> >>>>>>>>>>>>         being that
>> >> >>>>>>>>>>>>         > it
>> >> >>>>>>>>>>>>         > >>>>> can
>> >> >>>>>>>>>>>>         > >>>>> only resolve out-of-order arrival for up
>> to
>> >> the
>> >> >>>>> size of
>> >> >>>>>>>>>>>>         the window
>> >> >>>>>>>>>>>>         > (ie:
>> >> >>>>>>>>>>>>         > >>>>> 24h, 72h, etc). This would remove the
>> >> possibility
>> >> >>>>> of it
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> being
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > unbounded
>> >> >>>>>>>>>>>>         > >>>>> in
>> >> >>>>>>>>>>>>         > >>>>> size.
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>> With regards to Jan's suggestion, I
>> believe
>> >> this
>> >> >> is
>> >> >>>>> where
>> >> >>>>>>>>>>>>         we will
>> >> >>>>>>>>>>>>         > have
>> >> >>>>>>>>>>>>         > >>>>> to
>> >> >>>>>>>>>>>>         > >>>>> remain in disagreement. While I do not
>> >> disagree
>> >> >>>>> with your
>> >> >>>>>>>>>>>>         statement
>> >> >>>>>>>>>>>>         > >>>>> about
>> >> >>>>>>>>>>>>         > >>>>> there likely to be additional joins done
>> in a
>> >> >>>>> real-world
>> >> >>>>>>>>>>>>         workflow, I
>> >> >>>>>>>>>>>>         > do
>> >> >>>>>>>>>>>>         > >>>>> not
>> >> >>>>>>>>>>>>         > >>>>> see how you can conclusively deal with
>> >> >> out-of-order
>> >> >>>>>>>>>>>> arrival
>> >> >>>>>>>>>>>> of
>> >> >>>>>>>>>>>>         > >>>>> foreign-key
>> >> >>>>>>>>>>>>         > >>>>> changes and subsequent joins. I have
>> >> attempted
>> >> >> what
>> >> >>>>> I
>> >> >>>>>>>>>>>>         think you have
>> >> >>>>>>>>>>>>         > >>>>> proposed (without a high-water, using
>> >> groupBy and
>> >> >>>>> reduce)
>> >> >>>>>>>>>>>>         and found
>> >> >>>>>>>>>>>>         > >>>>> that if
>> >> >>>>>>>>>>>>         > >>>>> the foreign key changes too quickly, or
>> the
>> >> load
>> >> >> on
>> >> >>>>> a
>> >> >>>>>>>>>>>>         stream thread
>> >> >>>>>>>>>>>>         > is
>> >> >>>>>>>>>>>>         > >>>>> too
>> >> >>>>>>>>>>>>         > >>>>> high, the joined messages will arrive
>> >> >> out-of-order
>> >> >>>>> and be
>> >> >>>>>>>>>>>>         incorrectly
>> >> >>>>>>>>>>>>         > >>>>> propagated, such that an intermediate
>> event
>> >> is
>> >> >>>>>>>>>>>> represented
>> >> >>>>>>>>>>>>         as the
>> >> >>>>>>>>>>>>         > final
>> >> >>>>>>>>>>>>         > >>>>> event.
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>> Can you shed some light on your groupBy
>> >> >>>>> implementation.
>> >> >>>>>>>>>>>>         There must be
>> >> >>>>>>>>>>>>         > >>>> some sort of flaw in it.
>> >> >>>>>>>>>>>>         > >>>> I have a suspicion where it is, I would
>> just
>> >> like
>> >> >> to
>> >> >>>>>>>>>>>>         confirm. The idea
>> >> >>>>>>>>>>>>         > >>>> is bullet proof and it must be
>> >> >>>>>>>>>>>>         > >>>> an implementation mess up. I would like to
>> >> clarify
>> >> >>>>> before
>> >> >>>>>>>>>>>>         we draw a
>> >> >>>>>>>>>>>>         > >>>> conclusion.
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>    Repartitioning the scattered events
>> back to
>> >> >> their
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> original
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > >>>>> partitions is the only way I know how to
>> >> >> conclusively
>> >> >>>>> deal
>> >> >>>>>>>>>>>>         with
>> >> >>>>>>>>>>>>         > >>>>> out-of-order events in a given time frame,
>> >> and to
>> >> >>>>> ensure
>> >> >>>>>>>>>>>>         that the
>> >> >>>>>>>>>>>>         > data
>> >> >>>>>>>>>>>>         > >>>>> is
>> >> >>>>>>>>>>>>         > >>>>> eventually consistent with the input
>> events.
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>> If you have some code to share that
>> >> illustrates
>> >> >> your
>> >> >>>>>>>>>>>>         approach, I
>> >> >>>>>>>>>>>>         > would
>> >> >>>>>>>>>>>>         > >>>>> be
>> >> >>>>>>>>>>>>         > >>>>> very grateful as it would remove any
>> >> >>>>> misunderstandings
>> >> >>>>>>>>>>>>         that I may
>> >> >>>>>>>>>>>>         > have.
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>> ah okay you were looking for my code. I
>> don't
>> >> have
>> >> >>>>>>>>>>>>         something easily
>> >> >>>>>>>>>>>>         > >>>> readable here as its bloated with
>> OO-patterns.
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>> its anyhow trivial:
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>> @Override
>> >> >>>>>>>>>>>>         > >>>>      public T apply(K aggKey, V value, T
>> >> >> aggregate)
>> >> >>>>>>>>>>>>         > >>>>      {
>> >> >>>>>>>>>>>>         > >>>>          Map<U, V> currentStateAsMap =
>> >> >>>>> asMap(aggregate);
>> >> >>>>>>>>>>>> <<
>> >> >>>>>>>>>>>>         imaginary
>> >> >>>>>>>>>>>>         > >>>>          U toModifyKey =
>> mapper.apply(value);
>> >> >>>>>>>>>>>>         > >>>>              << this is the place where
>> people
>> >> >>>>> actually
>> >> >>>>>>>>>>>>         gonna have
>> >> >>>>>>>>>>>>         > issues
>> >> >>>>>>>>>>>>         > >>>> and why you probably couldn't do it. we
>> would
>> >> need
>> >> >>>>> to find
>> >> >>>>>>>>>>>>         a solution
>> >> >>>>>>>>>>>>         > here.
>> >> >>>>>>>>>>>>         > >>>> I didn't realize that yet.
>> >> >>>>>>>>>>>>         > >>>>              << we propagate the field in
>> the
>> >> >>>>> joiner, so
>> >> >>>>>>>>>>>>         that we can
>> >> >>>>>>>>>>>>         > pick
>> >> >>>>>>>>>>>>         > >>>> it up in an aggregate. Probably you have
>> not
>> >> >> thought
>> >> >>>>> of
>> >> >>>>>>>>>>>>         this in your
>> >> >>>>>>>>>>>>         > >>>> approach right?
>> >> >>>>>>>>>>>>         > >>>>              << I am very open to find a
>> >> generic
>> >> >>>>> solution
>> >> >>>>>>>>>>>>         here. In my
>> >> >>>>>>>>>>>>         > >>>> honest opinion this is broken in
>> >> >> KTableImpl.GroupBy
>> >> >>>>> that
>> >> >>>>>>>>>>>> it
>> >> >>>>>>>>>>>>         looses
>> >> >>>>>>>>>>>>         > the keys
>> >> >>>>>>>>>>>>         > >>>> and only maintains the aggregate key.
>> >> >>>>>>>>>>>>         > >>>>              << I abstracted it away back
>> >> then way
>> >> >>>>> before
>> >> >>>>>>>>>>>> i
>> >> >>>>>>>>>>>> was
>> >> >>>>>>>>>>>>         > thinking
>> >> >>>>>>>>>>>>         > >>>> of oneToMany join. That is why I didn't
>> >> realize
>> >> >> its
>> >> >>>>>>>>>>>>         significance here.
>> >> >>>>>>>>>>>>         > >>>>              << Opinions?
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>          for (V m : current)
>> >> >>>>>>>>>>>>         > >>>>          {
>> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(mapper.apply(m), m);
>> >> >>>>>>>>>>>>         > >>>>          }
>> >> >>>>>>>>>>>>         > >>>>          if (isAdder)
>> >> >>>>>>>>>>>>         > >>>>          {
>> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(toModifyKey, value);
>> >> >>>>>>>>>>>>         > >>>>          }
>> >> >>>>>>>>>>>>         > >>>>          else
>> >> >>>>>>>>>>>>         > >>>>          {
>> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.remove(toModifyKey);
>> >> >>>>>>>>>>>>         > >>>> if(currentStateAsMap.isEmpty()){
>> >> >>>>>>>>>>>>         > >>>>                  return null;
>> >> >>>>>>>>>>>>         > >>>>              }
>> >> >>>>>>>>>>>>         > >>>>          }
>> >> >>>>>>>>>>>>         > >>>>          retrun
>> >> asAggregateType(currentStateAsMap)
>> >> >>>>>>>>>>>>         > >>>>      }
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>>
>> >> >>>>>>>>>>>>         > >>>> Thanks,
>> >> >>>>>>>>>>>>         > >>>>> Adam
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
>> Filipiak
>> >> <
>> >> >>>>>>>>>>>>         > Jan.Filipiak@trivago.com <mailto:
>> >> >> Jan.Filipiak@trivago.com
>> >> >>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>> wrote:
>> >> >>>>>>>>>>>>         > >>>>>
>> >> >>>>>>>>>>>>         > >>>>> Thanks Adam for bringing Matthias to
>> speed!
>> >> >>>>>>>>>>>>         > >>>>>> about the differences. I think re-keying
>> >> back
>> >> >>>>> should be
>> >> >>>>>>>>>>>>         optional at
>> >> >>>>>>>>>>>>         > >>>>>> best.
>> >> >>>>>>>>>>>>         > >>>>>> I would say we return a KScatteredTable
>> with
>> >> >>>>> reshuffle()
>> >> >>>>>>>>>>>>         returning
>> >> >>>>>>>>>>>>         > >>>>>> KTable<originalKey,Joined> to make the
>> >> backwards
>> >> >>>>>>>>>>>>         repartitioning
>> >> >>>>>>>>>>>>         > >>>>>> optional.
>> >> >>>>>>>>>>>>         > >>>>>> I am also in a big favour of doing the
>> out
>> >> of
>> >> >> order
>> >> >>>>>>>>>>>>         processing using
>> >> >>>>>>>>>>>>         > >>>>>> group
>> >> >>>>>>>>>>>>         > >>>>>> by instead high water mark tracking.
>> >> >>>>>>>>>>>>         > >>>>>> Just because unbounded growth is just
>> scary
>> >> + It
>> >> >>>>> saves
>> >> >>>>>>>>>>>> us
>> >> >>>>>>>>>>>>         the header
>> >> >>>>>>>>>>>>         > >>>>>> stuff.
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>> I think the abstraction of always
>> >> repartitioning
>> >> >>>>> back is
>> >> >>>>>>>>>>>>         just not so
>> >> >>>>>>>>>>>>         > >>>>>> strong. Like the work has been done
>> before
>> >> we
>> >> >>>>> partition
>> >> >>>>>>>>>>>>         back and
>> >> >>>>>>>>>>>>         > >>>>>> grouping
>> >> >>>>>>>>>>>>         > >>>>>> by something else afterwards is really
>> >> common.
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>> On 05.09.2018 13:49, Adam Bellemare
>> wrote:
>> >> >>>>>>>>>>>>         > >>>>>>
>> >> >>>>>>>>>>>>         > >>>>>> Hi Matthias
>> >> >>>>>>>>>>>>         > >>>>>>> Thank you for your feedback, I do
>> >> appreciate
>> >> >> it!
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> While name spacing would be possible, it
>> >> would
>> >> >>>>> require
>> >> >>>>>>>>>>>> to
>> >> >>>>>>>>>>>>         > deserialize
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
>> >> overhead.
>> >> >> I
>> >> >>>>> would
>> >> >>>>>>>>>>>>         suggest to
>> >> >>>>>>>>>>>>         > no
>> >> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the
>> overhead.
>> >> If
>> >> >> this
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> becomes a
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > problem in
>> >> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name
>> spacing
>> >> >> later
>> >> >>>>> on.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Agreed. I will go with using a reserved
>> >> string
>> >> >>>>> and
>> >> >>>>>>>>>>>>         document it.
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> My main concern about the design it the
>> >> type of
>> >> >>>>> the
>> >> >>>>>>>>>>>>         result KTable:
>> >> >>>>>>>>>>>>         > If
>> >> >>>>>>>>>>>>         > >>>>>>> I
>> >> >>>>>>>>>>>>         > >>>>>>> understood the proposal correctly,
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> In your example, you have table1 and
>> table2
>> >> >>>>> swapped.
>> >> >>>>>>>>>>>>         Here is how it
>> >> >>>>>>>>>>>>         > >>>>>>> works
>> >> >>>>>>>>>>>>         > >>>>>>> currently:
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> 1) table1 has the records that contain
>> the
>> >> >>>>> foreign key
>> >> >>>>>>>>>>>>         within their
>> >> >>>>>>>>>>>>         > >>>>>>> value.
>> >> >>>>>>>>>>>>         > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>> >> >>>>>>>>>>>> <b,(fk=A,bar=2)>,
>> >> >>>>>>>>>>>>         > >>>>>>> <c,(fk=B,bar=3)>
>> >> >>>>>>>>>>>>         > >>>>>>> table2 input stream: <A,X>, <B,Y>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> 2) A Value mapper is required to extract
>> >> the
>> >> >>>>> foreign
>> >> >>>>>>>>>>>> key.
>> >> >>>>>>>>>>>>         > >>>>>>> table1 foreign key mapper: ( value =>
>> >> value.fk
>> >> >>>>>>>>>>>>         <http://value.fk> )
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> The mapper is applied to each element in
>> >> >> table1,
>> >> >>>>> and a
>> >> >>>>>>>>>>>>         new combined
>> >> >>>>>>>>>>>>         > >>>>>>> key is
>> >> >>>>>>>>>>>>         > >>>>>>> made:
>> >> >>>>>>>>>>>>         > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>,
>> <A-b,
>> >> >>>>>>>>>>>> (fk=A,bar=2)>,
>> >> >>>>>>>>>>>>         <B-c,
>> >> >>>>>>>>>>>>         > >>>>>>> (fk=B,bar=3)>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> 3) The rekeyed events are copartitioned
>> >> with
>> >> >>>>> table2:
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> a) Stream Thread with Partition 0:
>> >> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <A-a,
>> (fk=A,bar=1)>,
>> >> <A-b,
>> >> >>>>>>>>>>>>         (fk=A,bar=2)>
>> >> >>>>>>>>>>>>         > >>>>>>> Table2: <A,X>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> b) Stream Thread with Partition 1:
>> >> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>> >> >>>>>>>>>>>>         > >>>>>>> Table2: <B,Y>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> 4) From here, they can be joined
>> together
>> >> >> locally
>> >> >>>>> by
>> >> >>>>>>>>>>>>         applying the
>> >> >>>>>>>>>>>>         > >>>>>>> joiner
>> >> >>>>>>>>>>>>         > >>>>>>> function.
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> At this point, Jan's design and my
>> design
>> >> >>>>> deviate. My
>> >> >>>>>>>>>>>>         design goes
>> >> >>>>>>>>>>>>         > on
>> >> >>>>>>>>>>>>         > >>>>>>> to
>> >> >>>>>>>>>>>>         > >>>>>>> repartition the data post-join and
>> resolve
>> >> >>>>> out-of-order
>> >> >>>>>>>>>>>>         arrival of
>> >> >>>>>>>>>>>>         > >>>>>>> records,
>> >> >>>>>>>>>>>>         > >>>>>>> finally returning the data keyed just
>> the
>> >> >>>>> original key.
>> >> >>>>>>>>>>>>         I do not
>> >> >>>>>>>>>>>>         > >>>>>>> expose
>> >> >>>>>>>>>>>>         > >>>>>>> the
>> >> >>>>>>>>>>>>         > >>>>>>> CombinedKey or any of the internals
>> >> outside of
>> >> >> the
>> >> >>>>>>>>>>>>         joinOnForeignKey
>> >> >>>>>>>>>>>>         > >>>>>>> function. This does make for larger
>> >> footprint,
>> >> >>>>> but it
>> >> >>>>>>>>>>>>         removes all
>> >> >>>>>>>>>>>>         > >>>>>>> agency
>> >> >>>>>>>>>>>>         > >>>>>>> for resolving out-of-order arrivals and
>> >> >> handling
>> >> >>>>>>>>>>>>         CombinedKeys from
>> >> >>>>>>>>>>>>         > the
>> >> >>>>>>>>>>>>         > >>>>>>> user. I believe that this makes the
>> >> function
>> >> >> much
>> >> >>>>>>>>>>>> easier
>> >> >>>>>>>>>>>>         to use.
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> Let me know if this helps resolve your
>> >> >> questions,
>> >> >>>>> and
>> >> >>>>>>>>>>>>         please feel
>> >> >>>>>>>>>>>>         > >>>>>>> free to
>> >> >>>>>>>>>>>>         > >>>>>>> add anything else on your mind.
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> Thanks again,
>> >> >>>>>>>>>>>>         > >>>>>>> Adam
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM,
>> Matthias J.
>> >> >> Sax <
>> >> >>>>>>>>>>>>         > >>>>>>> matthias@confluent.io <mailto:
>> >> >>>>> matthias@confluent.io>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> wrote:
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>> Hi,
>> >> >>>>>>>>>>>>         > >>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> I am just catching up on this thread. I
>> >> did
>> >> >> not
>> >> >>>>> read
>> >> >>>>>>>>>>>>         everything so
>> >> >>>>>>>>>>>>         > >>>>>>>> far,
>> >> >>>>>>>>>>>>         > >>>>>>>> but want to share couple of initial
>> >> thoughts:
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Headers: I think there is a fundamental
>> >> >>>>> difference
>> >> >>>>>>>>>>>>         between header
>> >> >>>>>>>>>>>>         > >>>>>>>> usage
>> >> >>>>>>>>>>>>         > >>>>>>>> in this KIP and KP-258. For 258, we add
>> >> >> headers
>> >> >>>>> to
>> >> >>>>>>>>>>>>         changelog topic
>> >> >>>>>>>>>>>>         > >>>>>>>> that
>> >> >>>>>>>>>>>>         > >>>>>>>> are owned by Kafka Streams and nobody
>> >> else is
>> >> >>>>> supposed
>> >> >>>>>>>>>>>>         to write
>> >> >>>>>>>>>>>>         > into
>> >> >>>>>>>>>>>>         > >>>>>>>> them. In fact, no user header are
>> written
>> >> into
>> >> >>>>> the
>> >> >>>>>>>>>>>>         changelog topic
>> >> >>>>>>>>>>>>         > >>>>>>>> and
>> >> >>>>>>>>>>>>         > >>>>>>>> thus, there are not conflicts.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Nevertheless, I don't see a big issue
>> with
>> >> >> using
>> >> >>>>>>>>>>>>         headers within
>> >> >>>>>>>>>>>>         > >>>>>>>> Streams.
>> >> >>>>>>>>>>>>         > >>>>>>>> As long as we document it, we can have
>> >> some
>> >> >>>>> "reserved"
>> >> >>>>>>>>>>>>         header keys
>> >> >>>>>>>>>>>>         > >>>>>>>> and
>> >> >>>>>>>>>>>>         > >>>>>>>> users are not allowed to use when
>> >> processing
>> >> >>>>> data with
>> >> >>>>>>>>>>>>         Kafka
>> >> >>>>>>>>>>>>         > Streams.
>> >> >>>>>>>>>>>>         > >>>>>>>> IMHO, this should be ok.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> I think there is a safe way to avoid
>> >> >> conflicts,
>> >> >>>>> since
>> >> >>>>>>>>>>>> these
>> >> >>>>>>>>>>>>         > headers
>> >> >>>>>>>>>>>>         > >>>>>>>> are
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>> only needed in internal topics (I
>> think):
>> >> >>>>>>>>>>>>         > >>>>>>>>> For internal and changelog topics, we
>> can
>> >> >>>>> namespace
>> >> >>>>>>>>>>>>         all headers:
>> >> >>>>>>>>>>>>         > >>>>>>>>> * user-defined headers are namespaced
>> as
>> >> >>>>> "external."
>> >> >>>>>>>>>>>> +
>> >> >>>>>>>>>>>>         headerKey
>> >> >>>>>>>>>>>>         > >>>>>>>>> * internal headers are namespaced as
>> >> >>>>> "internal." +
>> >> >>>>>>>>>>>>         headerKey
>> >> >>>>>>>>>>>>         > >>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>> While name spacing would be possible,
>> it
>> >> >> would
>> >> >>>>>>>>>>>> require
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> to
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > >>>>>>>> deserialize
>> >> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
>> >> overhead.
>> >> >> I
>> >> >>>>> would
>> >> >>>>>>>>>>>>         suggest to
>> >> >>>>>>>>>>>>         > no
>> >> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the
>> overhead.
>> >> If
>> >> >> this
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> becomes a
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > problem in
>> >> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name
>> spacing
>> >> >> later
>> >> >>>>> on.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> My main concern about the design it the
>> >> type
>> >> >> of
>> >> >>>>> the
>> >> >>>>>>>>>>>>         result KTable:
>> >> >>>>>>>>>>>>         > >>>>>>>> If I
>> >> >>>>>>>>>>>>         > >>>>>>>> understood the proposal correctly,
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V1> table1 = ...
>> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K2,V2> table2 = ...
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V3> joinedTable =
>> >> >>>>> table1.join(table2,...);
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> implies that the `joinedTable` has the
>> >> same
>> >> >> key
>> >> >>>>> as the
>> >> >>>>>>>>>>>>         left input
>> >> >>>>>>>>>>>>         > >>>>>>>> table.
>> >> >>>>>>>>>>>>         > >>>>>>>> IMHO, this does not work because if
>> table2
>> >> >>>>> contains
>> >> >>>>>>>>>>>>         multiple rows
>> >> >>>>>>>>>>>>         > >>>>>>>> that
>> >> >>>>>>>>>>>>         > >>>>>>>> join with a record in table1 (what is
>> the
>> >> main
>> >> >>>>> purpose
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> of
>> >> >>>>>>>>>
>> >> >>>>>>>>>> a
>> >> >>>>>>>>>>>>         > foreign
>> >> >>>>>>>>>>>>         > >>>>>>>> key
>> >> >>>>>>>>>>>>         > >>>>>>>> join), the result table would only
>> >> contain a
>> >> >>>>> single
>> >> >>>>>>>>>>>>         join result,
>> >> >>>>>>>>>>>>         > but
>> >> >>>>>>>>>>>>         > >>>>>>>> not
>> >> >>>>>>>>>>>>         > >>>>>>>> multiple.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Example:
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> table1 input stream: <A,X>
>> >> >>>>>>>>>>>>         > >>>>>>>> table2 input stream: <a,(A,1)>,
>> <b,(A,2)>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> We use table2 value a foreign key to
>> >> table1
>> >> >> key
>> >> >>>>> (ie,
>> >> >>>>>>>>>>>>         "A" joins).
>> >> >>>>>>>>>>>>         > If
>> >> >>>>>>>>>>>>         > >>>>>>>> the
>> >> >>>>>>>>>>>>         > >>>>>>>> result key is the same key as key of
>> >> table1,
>> >> >> this
>> >> >>>>>>>>>>>>         implies that the
>> >> >>>>>>>>>>>>         > >>>>>>>> result can either be <A, join(X,1)> or
>> <A,
>> >> >>>>> join(X,2)>
>> >> >>>>>>>>>>>>         but not
>> >> >>>>>>>>>>>>         > both.
>> >> >>>>>>>>>>>>         > >>>>>>>> Because the share the same key,
>> whatever
>> >> >> result
>> >> >>>>> record
>> >> >>>>>>>>>>>>         we emit
>> >> >>>>>>>>>>>>         > later,
>> >> >>>>>>>>>>>>         > >>>>>>>> overwrite the previous result.
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> This is the reason why Jan originally
>> >> proposed
>> >> >>>>> to use
>> >> >>>>>>>>>>>> a
>> >> >>>>>>>>>>>>         > combination
>> >> >>>>>>>>>>>>         > >>>>>>>> of
>> >> >>>>>>>>>>>>         > >>>>>>>> both primary keys of the input tables
>> as
>> >> key
>> >> >> of
>> >> >>>>> the
>> >> >>>>>>>>>>>>         output table.
>> >> >>>>>>>>>>>>         > >>>>>>>> This
>> >> >>>>>>>>>>>>         > >>>>>>>> makes the keys of the output table
>> unique
>> >> and
>> >> >> we
>> >> >>>>> can
>> >> >>>>>>>>>>>>         store both in
>> >> >>>>>>>>>>>>         > >>>>>>>> the
>> >> >>>>>>>>>>>>         > >>>>>>>> output table:
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>> >> >>>>> join(X,2)>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Thoughts?
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> -Matthias
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>> >> >>>>>>>>>>>>         > >>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>> Just on remark here.
>> >> >>>>>>>>>>>>         > >>>>>>>>> The high-watermark could be
>> disregarded.
>> >> The
>> >> >>>>> decision
>> >> >>>>>>>>>>>>         about the
>> >> >>>>>>>>>>>>         > >>>>>>>>> forward
>> >> >>>>>>>>>>>>         > >>>>>>>>> depends on the size of the aggregated
>> >> map.
>> >> >>>>>>>>>>>>         > >>>>>>>>> Only 1 element long maps would be
>> >> unpacked
>> >> >> and
>> >> >>>>>>>>>>>>         forwarded. 0
>> >> >>>>>>>>>>>>         > element
>> >> >>>>>>>>>>>>         > >>>>>>>>> maps
>> >> >>>>>>>>>>>>         > >>>>>>>>> would be published as delete. Any
>> other
>> >> count
>> >> >>>>>>>>>>>>         > >>>>>>>>> of map entries is in "waiting for
>> correct
>> >> >>>>> deletes to
>> >> >>>>>>>>>>>>         > arrive"-state.
>> >> >>>>>>>>>>>>         > >>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
>> >> wrote:
>> >> >>>>>>>>>>>>         > >>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>> It does look like I could replace the
>> >> second
>> >> >>>>>>>>>>>>         repartition store
>> >> >>>>>>>>>>>>         > and
>> >> >>>>>>>>>>>>         > >>>>>>>>>> highwater store with a groupBy and
>> >> reduce.
>> >> >>>>> However,
>> >> >>>>>>>>>>>>         it looks
>> >> >>>>>>>>>>>>         > like
>> >> >>>>>>>>>>>>         > >>>>>>>>>> I
>> >> >>>>>>>>>>>>         > >>>>>>>>>> would
>> >> >>>>>>>>>>>>         > >>>>>>>>>> still need to store the highwater
>> value
>> >> >> within
>> >> >>>>> the
>> >> >>>>>>>>>>>>         materialized
>> >> >>>>>>>>>>>>         > >>>>>>>>>> store,
>> >> >>>>>>>>>>>>         > >>>>>>>>>>
>> >> >>>>>>>>>>>>         > >>>>>>>>>> to
>> >> >>>>>>>>>>>>         > >>>>>>>>> compare the arrival of out-of-order
>> >> records
>> >> >>>>> (assuming
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>> my
>> >> >>>>>>>>>
>> >> >>>>>>>>>>         > >>>>>>>>> understanding
>> >> >>>>>>>>>>>>         > >>>>>>>>> of
>> >> >>>>>>>>>>>>         > >>>>>>>>> THIS is correct...). This in effect is
>> >> the
>> >> >> same
>> >> >>>>> as
>> >> >>>>>>>>>>>> the
>> >> >>>>>>>>>>>>         design I
>> >> >>>>>>>>>>>>         > have
>> >> >>>>>>>>>>>>         > >>>>>>>>> now,
>> >> >>>>>>>>>>>>         > >>>>>>>>> just with the two tables merged
>> together.
>> >> >>>>>>>>>>>>         > >>>>>>>>>
>> >> >>>>>>>>>>>>         >
>> >> >>>>>>>>>>>>         >
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>>
>> >> >>>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> --
>> >> >>>>>>>>>> -- Guozhang
>> >> >>>>>>>>>>
>> >> >>>>>>>>>>
>> >> >>>>>>>>>
>> >> >>>>>>>>>
>> >> >>>>>>>>> --
>> >> >>>>>>>>> -- Guozhang
>> >> >>>>>>>>>
>> >> >>>>>>>>>
>> >> >>>>>>>>
>> >> >>>>>>>
>> >> >>>>>>
>> >> >>>>>
>> >> >>>>
>> >> >>>
>> >> >>
>> >> >
>> >>
>> >
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi John

Thanks for your feedback and assistance. I think your summary is accurate
from my perspective. Additionally, I would like to add that there is a risk
of inconsistent final states without performing the resolution. This is a
major concern for me as most of the data I have dealt with is produced by
relational databases. We have seen a number of cases where a user in the
Rails UI has modified the field (foreign key), realized they made a
mistake, and then updated the field again with a new key. The events are
propagated out as they are produced, and as such we have had real-world
cases where these inconsistencies were propagated downstream as the final
values due to the race conditions in the fanout of the data.

This solution that I propose values correctness of the final result over
other factors.

We could always move this function over to using a KScatteredTable
implementation in the future, and simply deprecate it this join API in
time. I think I would like to hear more from some of the other major
committers on which course of action they would think is best before any
more coding is done.

Thanks again

Adam


On Mon, Dec 3, 2018 at 8:24 PM John Roesler <jo...@confluent.io> wrote:

> Hi Jan and Adam,
>
> Wow, thanks for doing that test, Adam. Those results are encouraging.
>
> Thanks for your performance experience as well, Jan. I agree that avoiding
> unnecessary join outputs is especially important when the fan-out is so
> high. I suppose this could also be built into the implementation we're
> discussing, but it wouldn't have to be specified in the KIP (since it's an
> API-transparent optimization).
>
> As far as whether or not to re-repartition the data, I didn't bring it up
> because it sounded like the two of you agreed to leave the KIP as-is,
> despite the disagreement.
>
> If you want my opinion, I feel like both approaches are reasonable.
> It sounds like Jan values more the potential for developers to optimize
> their topologies to re-use the intermediate nodes, whereas Adam places more
> value on having a single operator that people can use without extra steps
> at the end.
>
> Personally, although I do find it exceptionally annoying when a framework
> gets in my way when I'm trying to optimize something, it seems better to go
> for a single operation.
> * Encapsulating the internal transitions gives us significant latitude in
> the implementation (for example, joining only at the end, not in the middle
> to avoid extra data copying and out-of-order resolution; how we represent
> the first repartition keys (combined keys vs. value vectors), etc.). If we
> publish something like a KScatteredTable with the right-partitioned joined
> data, then the API pretty much locks in the implementation as well.
> * The API seems simpler to understand and use. I do mean "seems"; if anyone
> wants to make the case that KScatteredTable is actually simpler, I think
> hypothetical usage code would help. From a relational algebra perspective,
> it seems like KTable.join(KTable) should produce a new KTable in all cases.
> * That said, there might still be room in the API for a different operation
> like what Jan has proposed to scatter a KTable, and then do things like
> join, re-group, etc from there... I'm not sure; I haven't thought through
> all the consequences yet.
>
> This is all just my opinion after thinking over the discussion so far...
> -John
>
> On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > Updated the PR to take into account John's feedback.
> >
> > I did some preliminary testing for the performance of the prefixScan. I
> > have attached the file, but I will also include the text in the body here
> > for archival purposes (I am not sure what happens to attached files). I
> > also updated the PR and the KIP accordingly.
> >
> > Summary: It scales exceptionally well for scanning large values of
> > records. As Jan mentioned previously, the real issue would be more around
> > processing the resulting records after obtaining them. For instance, it
> > takes approximately ~80-120 mS to flush the buffer and a further ~35-85mS
> > to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
> > through the records just to generate a simple count takes ~ 40 times
> longer
> > than the flush + scan combined.
> >
> >
> ============================================================================================
> > Setup:
> >
> >
> ============================================================================================
> > Java 9 with default settings aside from a 512 MB heap (Xmx512m, Xms512m)
> > CPU: i7 2.2 Ghz.
> >
> > Note: I am using a slightly-modified, directly-accessible Kafka Streams
> > RocksDB
> > implementation (RocksDB.java, basically just avoiding the
> > ProcessorContext).
> > There are no modifications to the default RocksDB values provided in the
> > 2.1/trunk release.
> >
> >
> > keysize = 128 bytes
> > valsize = 512 bytes
> >
> > Step 1:
> > Write X positive matching events: (key = prefix + left-padded
> > auto-incrementing integer)
> > Step 2:
> > Write 10X negative matching events (key = left-padded auto-incrementing
> > integer)
> > Step 3:
> > Perform flush
> > Step 4:
> > Perform prefixScan
> > Step 5:
> > Iterate through return Iterator and validate the count of expected
> events.
> >
> >
> >
> ============================================================================================
> > Results:
> >
> >
> ============================================================================================
> > X = 1k (11k events total)
> > Flush Time = 39 mS
> > Scan Time = 7 mS
> > 6.9 MB disk
> >
> >
> --------------------------------------------------------------------------------------------
> > X = 10k (110k events total)
> > Flush Time = 45 mS
> > Scan Time = 8 mS
> > 127 MB
> >
> >
> --------------------------------------------------------------------------------------------
> > X = 100k (1.1M events total)
> > Test1:
> > Flush Time = 60 mS
> > Scan Time = 12 mS
> > 678 MB
> >
> > Test2:
> > Flush Time = 45 mS
> > Scan Time = 7 mS
> > 576 MB
> >
> >
> --------------------------------------------------------------------------------------------
> > X = 1MB (11M events total)
> > Test1:
> > Flush Time = 52 mS
> > Scan Time = 19 mS
> > 7.2 GB
> >
> > Test2:
> > Flush Time = 84 mS
> > Scan Time = 34 mS
> > 9.1 GB
> >
> >
> --------------------------------------------------------------------------------------------
> > X = 2.5M (27.5M events total)
> > Test1:
> > Flush Time = 82 mS
> > Scan Time = 63 mS
> > 17GB - 276 sst files
> >
> > Test2:
> > Flush Time = 116 mS
> > Scan Time = 35 mS
> > 23GB - 361 sst files
> >
> > Test3:
> > Flush Time = 103 mS
> > Scan Time = 82 mS
> > 19 GB - 300 sst files
> >
> >
> --------------------------------------------------------------------------------------------
> >
> > I had to limit my testing on my laptop to X = 2.5M events. I tried to go
> > to X = 10M (110M events) but RocksDB was going into the 100GB+ range and
> my
> > laptop ran out of disk. More extensive testing could be done but I
> suspect
> > that it would be in line with what we're seeing in the results above.
> >
> >
> >
> >
> >
> >
> > At this point in time, I think the only major discussion point is really
> > around what Jan and I have disagreed on: repartitioning back + resolving
> > potential out of order issues or leaving that up to the client to handle.
> >
> >
> > Thanks folks,
> >
> > Adam
> >
> >
> > On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >>
> >>
> >> On 29.11.2018 15:14, John Roesler wrote:
> >> > Hi all,
> >> >
> >> > Sorry that this discussion petered out... I think the 2.1 release
> >> caused an
> >> > extended distraction that pushed it off everyone's radar (which was
> >> > precisely Adam's concern). Personally, I've also had some extend
> >> > distractions of my own that kept (and continue to keep) me
> preoccupied.
> >> >
> >> > However, calling for a vote did wake me up, so I guess Jan was on the
> >> right
> >> > track!
> >> >
> >> > I've gone back and reviewed the whole KIP document and the prior
> >> > discussion, and I'd like to offer a few thoughts:
> >> >
> >> > API Thoughts:
> >> >
> >> > 1. If I read the KIP right, you are proposing a many-to-one join.
> Could
> >> we
> >> > consider naming it manyToOneJoin? Or, if you prefer, flip the design
> >> around
> >> > and make it a oneToManyJoin?
> >> >
> >> > The proposed name "joinOnForeignKey" disguises the join type, and it
> >> seems
> >> > like it might trick some people into using it for a one-to-one join.
> >> This
> >> > would work, of course, but it would be super inefficient compared to a
> >> > simple rekey-and-join.
> >> >
> >> > 2. I might have missed it, but I don't think it's specified whether
> >> it's an
> >> > inner, outer, or left join. I'm guessing an outer join, as (neglecting
> >> IQ),
> >> > the rest can be achieved by filtering or by handling it in the
> >> ValueJoiner.
> >> >
> >> > 3. The arg list to joinOnForeignKey doesn't look quite right.
> >> > 3a. Regarding Serialized: There are a few different paradigms in play
> in
> >> > the Streams API, so it's confusing, but instead of three Serialized
> >> args, I
> >> > think it would be better to have one that allows (optionally) setting
> >> the 4
> >> > incoming serdes. The result serde is defined by the Materialized. The
> >> > incoming serdes can be optional because they might already be
> available
> >> on
> >> > the source KTables, or the default serdes from the config might be
> >> > applicable.
> >> >
> >> > 3b. Is the StreamPartitioner necessary? The other joins don't allow
> >> setting
> >> > one, and it seems like it might actually be harmful, since the rekey
> >> > operation needs to produce results that are co-partitioned with the
> >> "other"
> >> > KTable.
> >> >
> >> > 4. I'm fine with the "reserved word" header, but I didn't actually
> >> follow
> >> > what Matthias meant about namespacing requiring "deserializing" the
> >> record
> >> > header. The headers are already Strings, so I don't think that
> >> > deserialization is required. If we applied the namespace at source
> nodes
> >> > and stripped it at sink nodes, this would be practically no overhead.
> >> The
> >> > advantage of the namespace idea is that no public API change wrt
> headers
> >> > needs to happen, and no restrictions need to be placed on users'
> >> headers.
> >> >
> >> > (Although I'm wondering if we can get away without the header at
> all...
> >> > stay tuned)
> >> >
> >> > 5. I also didn't follow the discussion about the HWM table growing
> >> without
> >> > bound. As I read it, the HWM table is effectively implementing OCC to
> >> > resolve the problem you noted with disordering when the rekey is
> >> > reversed... particularly notable when the FK changes. As such, it only
> >> > needs to track the most recent "version" (the offset in the source
> >> > partition) of each key. Therefore, it should have the same number of
> >> keys
> >> > as the source table at all times.
> >> >
> >> > I see that you are aware of KIP-258, which I think might be relevant
> in
> >> a
> >> > couple of ways. One: it's just about storing the timestamp in the
> state
> >> > store, but the ultimate idea is to effectively use the timestamp as an
> >> OCC
> >> > "version" to drop disordered updates. You wouldn't want to use the
> >> > timestamp for this operation, but if you were to use a similar
> >> mechanism to
> >> > store the source offset in the store alongside the re-keyed values,
> then
> >> > you could avoid a separate table.
> >> >
> >> > 6. You and Jan have been thinking about this for a long time, so I've
> >> > probably missed something here, but I'm wondering if we can avoid the
> >> HWM
> >> > tracking at all and resolve out-of-order during a final join
> instead...
> >> >
> >> > Let's say we're joining a left table (Integer K: Letter FK, (other
> >> data))
> >> > to a right table (Letter K: (some data)).
> >> >
> >> > Left table:
> >> > 1: (A, xyz)
> >> > 2: (B, asd)
> >> >
> >> > Right table:
> >> > A: EntityA
> >> > B: EntityB
> >> >
> >> > We could do a rekey as you proposed with a combined key, but not
> >> > propagating the value at all..
> >> > Rekey table:
> >> > A-1: (dummy value)
> >> > B-2: (dummy value)
> >> >
> >> > Which we then join with the right table to produce:
> >> > A-1: EntityA
> >> > B-2: EntityB
> >> >
> >> > Which gets rekeyed back:
> >> > 1: A, EntityA
> >> > 2: B, EntityB
> >> >
> >> > And finally we do the actual join:
> >> > Result table:
> >> > 1: ((A, xyz), EntityA)
> >> > 2: ((B, asd), EntityB)
> >> >
> >> > The thing is that in that last join, we have the opportunity to
> compare
> >> the
> >> > current FK in the left table with the incoming PK of the right table.
> If
> >> > they don't match, we just drop the event, since it must be outdated.
> >> >
> >>
> >> > In your KIP, you gave an example in which (1: A, xyz) gets updated to
> >> (1:
> >> > B, xyz), ultimately yielding a conundrum about whether the final state
> >> > should be (1: null) or (1: joined-on-B). With the algorithm above, you
> >> > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> >> > EntityB)). It seems like this does give you enough information to make
> >> the
> >> > right choice, regardless of disordering.
> >>
> >> Will check Adams patch, but this should work. As mentioned often I am
> >> not convinced on partitioning back for the user automatically. I think
> >> this is the real performance eater ;)
> >>
> >> >
> >> >
> >> > 7. Last thought... I'm a little concerned about the performance of the
> >> > range scans when records change in the right table. You've said that
> >> you've
> >> > been using the algorithm you presented in production for a while. Can
> >> you
> >> > give us a sense of the performance characteristics you've observed?
> >> >
> >>
> >> Make it work, make it fast, make it beautiful. The topmost thing here is
> >> / was correctness. In practice I do not measure the performance of the
> >> range scan. Usual cases I run this with is emitting 500k - 1kk rows
> >> on a left hand side change. The range scan is just the work you gotta
> >> do, also when you pack your data into different formats, usually the
> >> rocks performance is very tight to the size of the data and we can't
> >> really change that. It is more important for users to prevent useless
> >> updates to begin with. My left hand side is guarded to drop changes that
> >> are not going to change my join output.
> >>
> >> usually it's:
> >>
> >> drop unused fields and then don't forward if old.equals(new)
> >>
> >> regarding to the performance of creating an iterator for smaller
> >> fanouts, users can still just do a group by first then anyways.
> >>
> >>
> >>
> >> > I could only think of one alternative, but I'm not sure if it's better
> >> or
> >> > worse... If the first re-key only needs to preserve the original key,
> >> as I
> >> > proposed in #6, then we could store a vector of keys in the value:
> >> >
> >> > Left table:
> >> > 1: A,...
> >> > 2: B,...
> >> > 3: A,...
> >> >
> >> > Gets re-keyed:
> >> > A: [1, 3]
> >> > B: [2]
> >> >
> >> > Then, the rhs part of the join would only need a regular single-key
> >> lookup.
> >> > Of course we have to deal with the problem of large values, as there's
> >> no
> >> > bound on the number of lhs records that can reference rhs records.
> >> Offhand,
> >> > I'd say we could page the values, so when one row is past the
> >> threshold, we
> >> > append the key for the next page. Then in most cases, it would be a
> >> single
> >> > key lookup, but for large fan-out updates, it would be one per (max
> >> value
> >> > size)/(avg lhs key size).
> >> >
> >> > This seems more complex, though... Plus, I think there's some extra
> >> > tracking we'd need to do to know when to emit a retraction. For
> example,
> >> > when record 1 is deleted, the re-key table would just have (A: [3]).
> >> Some
> >> > kind of tombstone is needed so that the join result for 1 can also be
> >> > retracted.
> >> >
> >> > That's all!
> >> >
> >> > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> >> > discussion has been slow.
> >> > -John
> >> >
> >> >
> >> > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <
> Jan.Filipiak@trivago.com>
> >> > wrote:
> >> >
> >> >> Id say you can just call the vote.
> >> >>
> >> >> that happens all the time, and if something comes up, it just goes
> back
> >> >> to discuss.
> >> >>
> >> >> would not expect to much attention with another another email in this
> >> >> thread.
> >> >>
> >> >> best Jan
> >> >>
> >> >> On 09.10.2018 13:56, Adam Bellemare wrote:
> >> >>> Hello Contributors
> >> >>>
> >> >>> I know that 2.1 is about to be released, but I do need to bump this
> to
> >> >> keep
> >> >>> visibility up. I am still intending to push this through once
> >> contributor
> >> >>> feedback is given.
> >> >>>
> >> >>> Main points that need addressing:
> >> >>> 1) Any way (or benefit) in structuring the current singular graph
> node
> >> >> into
> >> >>> multiple nodes? It has a whopping 25 parameters right now. I am a
> bit
> >> >> fuzzy
> >> >>> on how the optimizations are supposed to work, so I would appreciate
> >> any
> >> >>> help on this aspect.
> >> >>>
> >> >>> 2) Overall strategy for joining + resolving. This thread has much
> >> >> discourse
> >> >>> between Jan and I between the current highwater mark proposal and a
> >> >> groupBy
> >> >>> + reduce proposal. I am of the opinion that we need to strictly
> handle
> >> >> any
> >> >>> chance of out-of-order data and leave none of it up to the consumer.
> >> Any
> >> >>> comments or suggestions here would also help.
> >> >>>
> >> >>> 3) Anything else that you see that would prevent this from moving
> to a
> >> >> vote?
> >> >>>
> >> >>> Thanks
> >> >>>
> >> >>> Adam
> >> >>>
> >> >>>
> >> >>>
> >> >>>
> >> >>>
> >> >>>
> >> >>>
> >> >>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >> >> adam.bellemare@gmail.com>
> >> >>> wrote:
> >> >>>
> >> >>>> Hi Jan
> >> >>>>
> >> >>>> With the Stores.windowStoreBuilder and
> Stores.persistentWindowStore,
> >> you
> >> >>>> actually only need to specify the amount of segments you want and
> how
> >> >> large
> >> >>>> they are. To the best of my understanding, what happens is that the
> >> >>>> segments are automatically rolled over as new data with new
> >> timestamps
> >> >> are
> >> >>>> created. We use this exact functionality in some of the work done
> >> >>>> internally at my company. For reference, this is the hopping
> windowed
> >> >> store.
> >> >>>>
> >> >>>>
> >> >>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >> >>>>
> >> >>>> In the code that I have provided, there are going to be two 24h
> >> >> segments.
> >> >>>> When a record is put into the windowStore, it will be inserted at
> >> time
> >> >> T in
> >> >>>> both segments. The two segments will always overlap by 12h. As time
> >> >> goes on
> >> >>>> and new records are added (say at time T+12h+), the oldest segment
> >> will
> >> >> be
> >> >>>> automatically deleted and a new segment created. The records are by
> >> >> default
> >> >>>> inserted with the context.timestamp(), such that it is the record
> >> time,
> >> >> not
> >> >>>> the clock time, which is used.
> >> >>>>
> >> >>>> To the best of my understanding, the timestamps are retained when
> >> >>>> restoring from the changelog.
> >> >>>>
> >> >>>> Basically, this is heavy-handed way to deal with TTL at a
> >> segment-level,
> >> >>>> instead of at an individual record level.
> >> >>>>
> >> >>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> >> Jan.Filipiak@trivago.com>
> >> >>>> wrote:
> >> >>>>
> >> >>>>> Will that work? I expected it to blow up with ClassCastException
> or
> >> >>>>> similar.
> >> >>>>>
> >> >>>>> You either would have to specify the window you fetch/put or
> iterate
> >> >>>>> across all windows the key was found in right?
> >> >>>>>
> >> >>>>> I just hope the window-store doesn't check stream-time under the
> >> hoods
> >> >>>>> that would be a questionable interface.
> >> >>>>>
> >> >>>>> If it does: did you see my comment on checking all the windows
> >> earlier?
> >> >>>>> that would be needed to actually give reasonable time gurantees.
> >> >>>>>
> >> >>>>> Best
> >> >>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >> >>>>>> Hi Jan
> >> >>>>>>
> >> >>>>>> Check for  " highwaterMat " in the PR. I only changed the state
> >> store,
> >> >>>>> not
> >> >>>>>> the ProcessorSupplier.
> >> >>>>>>
> >> >>>>>> Thanks,
> >> >>>>>> Adam
> >> >>>>>>
> >> >>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >> >> Jan.Filipiak@trivago.com
> >> >>>>>>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >> >>>>>>>
> >> >>>>>>>> @Guozhang
> >> >>>>>>>>
> >> >>>>>>>> Thanks for the information. This is indeed something that will
> be
> >> >>>>>>>> extremely
> >> >>>>>>>> useful for this KIP.
> >> >>>>>>>>
> >> >>>>>>>> @Jan
> >> >>>>>>>> Thanks for your explanations. That being said, I will not be
> >> moving
> >> >>>>> ahead
> >> >>>>>>>> with an implementation using reshuffle/groupBy solution as you
> >> >>>>> propose.
> >> >>>>>>>> That being said, if you wish to implement it yourself off of my
> >> >>>>> current PR
> >> >>>>>>>> and submit it as a competitive alternative, I would be more
> than
> >> >>>>> happy to
> >> >>>>>>>> help vet that as an alternate solution. As it stands right now,
> >> I do
> >> >>>>> not
> >> >>>>>>>> really have more time to invest into alternatives without there
> >> >> being
> >> >>>>> a
> >> >>>>>>>> strong indication from the binding voters which they would
> >> prefer.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>> Hey, total no worries. I think I personally gave up on the
> streams
> >> >> DSL
> >> >>>>> for
> >> >>>>>>> some time already, otherwise I would have pulled this KIP
> through
> >> >>>>> already.
> >> >>>>>>> I am currently reimplementing my own DSL based on PAPI.
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>> I will look at finishing up my PR with the windowed state store
> >> in
> >> >> the
> >> >>>>>>>> next
> >> >>>>>>>> week or so, exercising it via tests, and then I will come back
> >> for
> >> >>>>> final
> >> >>>>>>>> discussions. In the meantime, I hope that any of the binding
> >> voters
> >> >>>>> could
> >> >>>>>>>> take a look at the KIP in the wiki. I have updated it according
> >> to
> >> >> the
> >> >>>>>>>> latest plan:
> >> >>>>>>>>
> >> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >> >>>>>>>> Support+non-key+joining+in+KTable
> >> >>>>>>>>
> >> >>>>>>>> I have also updated the KIP PR to use a windowed store. This
> >> could
> >> >> be
> >> >>>>>>>> replaced by the results of KIP-258 whenever they are completed.
> >> >>>>>>>> https://github.com/apache/kafka/pull/5527
> >> >>>>>>>>
> >> >>>>>>>> Thanks,
> >> >>>>>>>>
> >> >>>>>>>> Adam
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated
> in
> >> the
> >> >>>>> PR?
> >> >>>>>>> expected it to change to Windowed<K>,Long Missing something?
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >> >>>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link,
> >> as it
> >> >>>>> is
> >> >>>>>>>>> for
> >> >>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we
> do
> >> >>>>> want to
> >> >>>>>>>>> have "handling out-of-order data for source KTable" such that
> >> >>>>> instead of
> >> >>>>>>>>> blindly apply the updates to the materialized store, i.e.
> >> following
> >> >>>>>>>>> offset
> >> >>>>>>>>> ordering, we will reject updates that are older than the
> current
> >> >>>>> key's
> >> >>>>>>>>> timestamps, i.e. following timestamp ordering.
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> Guozhang
> >> >>>>>>>>>
> >> >>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> >> >> wangguoz@gmail.com>
> >> >>>>>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>> Hello Adam,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e.
> the
> >> >> high
> >> >>>>>>>>>> watermark store, now altered to be replaced with a window
> >> store),
> >> >> I
> >> >>>>>>>>>> think
> >> >>>>>>>>>> another current on-going KIP may actually help:
> >> >>>>>>>>>>
> >> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> >>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> This is for adding the timestamp into a key-value store (i.e.
> >> only
> >> >>>>> for
> >> >>>>>>>>>> non-windowed KTable), and then one of its usage, as described
> >> in
> >> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we
> >> can
> >> >>>>> then
> >> >>>>>>>>>> "reject" updates from the source topics if its timestamp is
> >> >> smaller
> >> >>>>> than
> >> >>>>>>>>>> the current key's latest update timestamp. I think it is very
> >> >>>>> similar to
> >> >>>>>>>>>> what you have in mind for high watermark based filtering,
> while
> >> >> you
> >> >>>>> only
> >> >>>>>>>>>> need to make sure that the timestamps of the joining records
> >> are
> >> >>>>>>>>>>
> >> >>>>>>>>> correctly
> >> >>>>>>>>>
> >> >>>>>>>>>> inherited though the whole topology to the final stage.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Note that this KIP is for key-value store and hence
> >> non-windowed
> >> >>>>> KTables
> >> >>>>>>>>>> only, but for windowed KTables we do not really have a good
> >> >> support
> >> >>>>> for
> >> >>>>>>>>>> their joins anyways (
> >> >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >> >>>>>>>>>> I
> >> >>>>>>>>>> think we can just consider non-windowed KTable-KTable non-key
> >> >> joins
> >> >>>>> for
> >> >>>>>>>>>> now. In which case, KIP-258 should help.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Guozhang
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >> >>>>> Jan.Filipiak@trivago.com
> >> >>>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Hi Guozhang
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Current highwater mark implementation would grow endlessly
> >> based
> >> >>>>> on
> >> >>>>>>>>>>>> primary key of original event. It is a pair of (<this table
> >> >>>>> primary
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> key>,
> >> >>>>>>>>>
> >> >>>>>>>>>> <highest offset seen for that key>). This is used to
> >> differentiate
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> between
> >> >>>>>>>>>
> >> >>>>>>>>>> late arrivals and new updates. My newest proposal would be to
> >> >>>>> replace
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> it
> >> >>>>>>>>>
> >> >>>>>>>>>> with a Windowed state store of Duration N. This would allow
> the
> >> >> same
> >> >>>>>>>>>>>> behaviour, but cap the size based on time. This should
> allow
> >> for
> >> >>>>> all
> >> >>>>>>>>>>>> late-arriving events to be processed, and should be
> >> customizable
> >> >>>>> by
> >> >>>>>>>>>>>> the
> >> >>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
> >> minutes
> >> >> of
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> window,
> >> >>>>>>>>>
> >> >>>>>>>>>> or perhaps 7 days...).
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Hi Adam, using time based retention can do the trick here.
> >> Even
> >> >>>>> if I
> >> >>>>>>>>>>> would still like to see the automatic repartitioning
> optional
> >> >>>>> since I
> >> >>>>>>>>>>>
> >> >>>>>>>>>> would
> >> >>>>>>>>>
> >> >>>>>>>>>> just reshuffle again. With windowed store I am a little bit
> >> >>>>> sceptical
> >> >>>>>>>>>>>
> >> >>>>>>>>>> about
> >> >>>>>>>>>
> >> >>>>>>>>>> how to determine the window. So esentially one could run into
> >> >>>>> problems
> >> >>>>>>>>>>>
> >> >>>>>>>>>> when
> >> >>>>>>>>>
> >> >>>>>>>>>> the rapid change happens near a window border. I will check
> you
> >> >>>>>>>>>>> implementation in detail, if its problematic, we could still
> >> >> check
> >> >>>>>>>>>>> _all_
> >> >>>>>>>>>>> windows on read with not to bad performance impact I guess.
> >> Will
> >> >>>>> let
> >> >>>>>>>>>>> you
> >> >>>>>>>>>>> know if the implementation would be correct as is. I
> wouldn't
> >> not
> >> >>>>> like
> >> >>>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>
> >> >>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> >> >> timestamp(B).
> >> >>>>> I
> >> >>>>>>>>>>>
> >> >>>>>>>>>> think
> >> >>>>>>>>>
> >> >>>>>>>>>> we can't expect that.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> @Jan
> >> >>>>>>>>>>>> I believe I understand what you mean now - thanks for the
> >> >>>>> diagram, it
> >> >>>>>>>>>>>> did really help. You are correct that I do not have the
> >> original
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> primary
> >> >>>>>>>>>
> >> >>>>>>>>>> key available, and I can see that if it was available then
> you
> >> >>>>> would be
> >> >>>>>>>>>>>> able to add and remove events from the Map. That being
> said,
> >> I
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> encourage
> >> >>>>>>>>>
> >> >>>>>>>>>> you to finish your diagrams / charts just for clarity for
> >> everyone
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> else.
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> >> >>>>> understand
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> the benefits for the rest. Sorry about the original primary
> >> key,
> >> >> We
> >> >>>>>>>>>>> have
> >> >>>>>>>>>>> join and Group by implemented our own in PAPI and basically
> >> not
> >> >>>>> using
> >> >>>>>>>>>>>
> >> >>>>>>>>>> any
> >> >>>>>>>>>
> >> >>>>>>>>>> DSL (Just the abstraction). Completely missed that in
> original
> >> DSL
> >> >>>>> its
> >> >>>>>>>>>>>
> >> >>>>>>>>>> not
> >> >>>>>>>>>
> >> >>>>>>>>>> there and just assumed it. total brain mess up on my end.
> Will
> >> >>>>> finish
> >> >>>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>
> >> >>>>>>>>>> chart as soon as i get a quite evening this week.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> My follow up question for you is, won't the Map stay inside
> >> the
> >> >>>>> State
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>> Store indefinitely after all of the changes have
> propagated?
> >> >> Isn't
> >> >>>>>>>>>>>> this
> >> >>>>>>>>>>>> effectively the same as a highwater mark state store?
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
> >> return
> >> >>>>> `null`
> >> >>>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>
> >> >>>>>>>>>> the key is removed from the keyspace. But there is going to
> be
> >> a
> >> >>>>> store
> >> >>>>>>>>>>> 100%, the good thing is that I can use this store directly
> for
> >> >>>>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
> >> >>>>> satisfying
> >> >>>>>>>>>>> all gurantees needed for further groupby / join. The
> Windowed
> >> >>>>> store is
> >> >>>>>>>>>>>
> >> >>>>>>>>>> not
> >> >>>>>>>>>
> >> >>>>>>>>>> keeping the values, so for the next statefull operation we
> >> would
> >> >>>>>>>>>>> need to instantiate an extra store. or we have the window
> >> store
> >> >>>>> also
> >> >>>>>>>>>>>
> >> >>>>>>>>>> have
> >> >>>>>>>>>
> >> >>>>>>>>>> the values then.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Long story short. if we can flip in a custom group by before
> >> >>>>>>>>>>> repartitioning to the original primary key i think it would
> >> help
> >> >>>>> the
> >> >>>>>>>>>>>
> >> >>>>>>>>>> users
> >> >>>>>>>>>
> >> >>>>>>>>>> big time in building efficient apps. Given the original
> primary
> >> >> key
> >> >>>>>>>>>>>
> >> >>>>>>>>>> issue I
> >> >>>>>>>>>
> >> >>>>>>>>>> understand that we do not have a solid foundation to build
> on.
> >> >>>>>>>>>>> Leaving primary key carry along to the user. very
> >> unfortunate. I
> >> >>>>> could
> >> >>>>>>>>>>> understand the decision goes like that. I do not think its a
> >> good
> >> >>>>>>>>>>>
> >> >>>>>>>>>> decision.
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Thanks
> >> >>>>>>>>>>>> Adam
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >> >>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> >> dumbreprajakta311@gmail.com
> >> >>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>>>         please remove me from this group
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> >> Jan.Filipiak@trivago.com
> >> >>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         wrote:
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > Hi Adam,
> >> >>>>>>>>>>>>         >
> >> >>>>>>>>>>>>         > give me some time, will make such a chart. last
> >> time i
> >> >>>>> didn't
> >> >>>>>>>>>>>>         get along
> >> >>>>>>>>>>>>         > well with giphy and ruined all your charts.
> >> >>>>>>>>>>>>         > Hopefully i can get it done today
> >> >>>>>>>>>>>>         >
> >> >>>>>>>>>>>>         > On 08.09.2018 16:00, Adam Bellemare wrote:
> >> >>>>>>>>>>>>         > > Hi Jan
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > > I have included a diagram of what I attempted
> on
> >> the
> >> >>>>> KIP.
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         >
> >> >>>>>>>>>>>>
> >> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >> >>>>>>>>>>>>
> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >> >>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >> >>>>>>>>>>>>         <
> >> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >> >>>>>>>>>>>>
> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >> >>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > > I attempted this back at the start of my own
> >> >>>>> implementation
> >> >>>>>>>>>>>> of
> >> >>>>>>>>>>>>         this
> >> >>>>>>>>>>>>         > > solution, and since I could not get it to work
> I
> >> have
> >> >>>>> since
> >> >>>>>>>>>>>>         discarded the
> >> >>>>>>>>>>>>         > > code. At this point in time, if you wish to
> >> continue
> >> >>>>> pursuing
> >> >>>>>>>>>>>>         for your
> >> >>>>>>>>>>>>         > > groupBy solution, I ask that you please create
> a
> >> >>>>> diagram on
> >> >>>>>>>>>>>>         the KIP
> >> >>>>>>>>>>>>         > > carefully explaining your solution. Please feel
> >> free
> >> >> to
> >> >>>>> use
> >> >>>>>>>>>>>>         the image I
> >> >>>>>>>>>>>>         > > just posted as a starting point. I am having
> >> trouble
> >> >>>>>>>>>>>>         understanding your
> >> >>>>>>>>>>>>         > > explanations but I think that a carefully
> >> constructed
> >> >>>>> diagram
> >> >>>>>>>>>>>>         will clear
> >> >>>>>>>>>>>>         > up
> >> >>>>>>>>>>>>         > > any misunderstandings. Alternately, please
> post a
> >> >>>>>>>>>>>>         comprehensive PR with
> >> >>>>>>>>>>>>         > > your solution. I can only guess at what you
> >> mean, and
> >> >>>>> since I
> >> >>>>>>>>>>>>         value my
> >> >>>>>>>>>>>>         > own
> >> >>>>>>>>>>>>         > > time as much as you value yours, I believe it
> is
> >> your
> >> >>>>>>>>>>>>         responsibility to
> >> >>>>>>>>>>>>         > > provide an implementation instead of me trying
> to
> >> >> guess.
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > > Adam
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> >> Jan.Filipiak@trivago.com
> >> >>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > > wrote:
> >> >>>>>>>>>>>>         > >
> >> >>>>>>>>>>>>         > >> Hi James,
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >> nice to see you beeing interested. kafka
> >> streams at
> >> >>>>> this
> >> >>>>>>>>>>>>         point supports
> >> >>>>>>>>>>>>         > >> all sorts of joins as long as both streams
> have
> >> the
> >> >>>>> same
> >> >>>>>>>>>>>> key.
> >> >>>>>>>>>>>>         > >> Adam is currently implementing a join where a
> >> KTable
> >> >>>>> and a
> >> >>>>>>>>>>>>         KTable can
> >> >>>>>>>>>>>>         > have
> >> >>>>>>>>>>>>         > >> a one to many relation ship (1:n). We exploit
> >> that
> >> >>>>> rocksdb
> >> >>>>>>>>>>>> is
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> a
> >> >>>>>>>>>
> >> >>>>>>>>>>         > >> datastore that keeps data sorted (At least
> >> exposes an
> >> >>>>> API to
> >> >>>>>>>>>>>>         access the
> >> >>>>>>>>>>>>         > >> stored data in a sorted fashion).
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >> I think the technical caveats are well
> >> understood
> >> >> now
> >> >>>>> and we
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> are
> >> >>>>>>>>>
> >> >>>>>>>>>>         > basically
> >> >>>>>>>>>>>>         > >> down to philosophy and API Design ( when Adam
> >> sees
> >> >> my
> >> >>>>> newest
> >> >>>>>>>>>>>>         message).
> >> >>>>>>>>>>>>         > >> I have a lengthy track record of loosing those
> >> kinda
> >> >>>>>>>>>>>>         arguments within
> >> >>>>>>>>>>>>         > the
> >> >>>>>>>>>>>>         > >> streams community and I have no clue why. So I
> >> >>>>> literally
> >> >>>>>>>>>>>>         can't wait for
> >> >>>>>>>>>>>>         > you
> >> >>>>>>>>>>>>         > >> to churn through this thread and give you
> >> opinion on
> >> >>>>> how we
> >> >>>>>>>>>>>>         should
> >> >>>>>>>>>>>>         > design
> >> >>>>>>>>>>>>         > >> the return type of the oneToManyJoin and how
> >> many
> >> >>>>> power we
> >> >>>>>>>>>>>>         want to give
> >> >>>>>>>>>>>>         > to
> >> >>>>>>>>>>>>         > >> the user vs "simplicity" (where simplicity
> isn't
> >> >>>>> really that
> >> >>>>>>>>>>>>         as users
> >> >>>>>>>>>>>>         > still
> >> >>>>>>>>>>>>         > >> need to understand it I argue)
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >> waiting for you to join in on the discussion
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >> Best Jan
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >> On 07.09.2018 15:49, James Kwan wrote:
> >> >>>>>>>>>>>>         > >>
> >> >>>>>>>>>>>>         > >>> I am new to this group and I found this
> subject
> >> >>>>>>>>>>>>         interesting.  Sounds
> >> >>>>>>>>>>>>         > like
> >> >>>>>>>>>>>>         > >>> you guys want to implement a join table of
> two
> >> >>>>> streams? Is
> >> >>>>>>>>>>>> there
> >> >>>>>>>>>>>>         > somewhere
> >> >>>>>>>>>>>>         > >>> I can see the original requirement or
> proposal?
> >> >>>>>>>>>>>>         > >>>
> >> >>>>>>>>>>>>         > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> >> Jan.Filipiak@trivago.com
> >> >>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > >>>> wrote:
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>> I'm currently testing using a Windowed
> Store
> >> to
> >> >>>>> store the
> >> >>>>>>>>>>>>         highwater
> >> >>>>>>>>>>>>         > >>>>> mark.
> >> >>>>>>>>>>>>         > >>>>> By all indications this should work fine,
> >> with
> >> >> the
> >> >>>>> caveat
> >> >>>>>>>>>>>>         being that
> >> >>>>>>>>>>>>         > it
> >> >>>>>>>>>>>>         > >>>>> can
> >> >>>>>>>>>>>>         > >>>>> only resolve out-of-order arrival for up to
> >> the
> >> >>>>> size of
> >> >>>>>>>>>>>>         the window
> >> >>>>>>>>>>>>         > (ie:
> >> >>>>>>>>>>>>         > >>>>> 24h, 72h, etc). This would remove the
> >> possibility
> >> >>>>> of it
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> being
> >> >>>>>>>>>
> >> >>>>>>>>>>         > unbounded
> >> >>>>>>>>>>>>         > >>>>> in
> >> >>>>>>>>>>>>         > >>>>> size.
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>> With regards to Jan's suggestion, I believe
> >> this
> >> >> is
> >> >>>>> where
> >> >>>>>>>>>>>>         we will
> >> >>>>>>>>>>>>         > have
> >> >>>>>>>>>>>>         > >>>>> to
> >> >>>>>>>>>>>>         > >>>>> remain in disagreement. While I do not
> >> disagree
> >> >>>>> with your
> >> >>>>>>>>>>>>         statement
> >> >>>>>>>>>>>>         > >>>>> about
> >> >>>>>>>>>>>>         > >>>>> there likely to be additional joins done
> in a
> >> >>>>> real-world
> >> >>>>>>>>>>>>         workflow, I
> >> >>>>>>>>>>>>         > do
> >> >>>>>>>>>>>>         > >>>>> not
> >> >>>>>>>>>>>>         > >>>>> see how you can conclusively deal with
> >> >> out-of-order
> >> >>>>>>>>>>>> arrival
> >> >>>>>>>>>>>> of
> >> >>>>>>>>>>>>         > >>>>> foreign-key
> >> >>>>>>>>>>>>         > >>>>> changes and subsequent joins. I have
> >> attempted
> >> >> what
> >> >>>>> I
> >> >>>>>>>>>>>>         think you have
> >> >>>>>>>>>>>>         > >>>>> proposed (without a high-water, using
> >> groupBy and
> >> >>>>> reduce)
> >> >>>>>>>>>>>>         and found
> >> >>>>>>>>>>>>         > >>>>> that if
> >> >>>>>>>>>>>>         > >>>>> the foreign key changes too quickly, or the
> >> load
> >> >> on
> >> >>>>> a
> >> >>>>>>>>>>>>         stream thread
> >> >>>>>>>>>>>>         > is
> >> >>>>>>>>>>>>         > >>>>> too
> >> >>>>>>>>>>>>         > >>>>> high, the joined messages will arrive
> >> >> out-of-order
> >> >>>>> and be
> >> >>>>>>>>>>>>         incorrectly
> >> >>>>>>>>>>>>         > >>>>> propagated, such that an intermediate event
> >> is
> >> >>>>>>>>>>>> represented
> >> >>>>>>>>>>>>         as the
> >> >>>>>>>>>>>>         > final
> >> >>>>>>>>>>>>         > >>>>> event.
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>> Can you shed some light on your groupBy
> >> >>>>> implementation.
> >> >>>>>>>>>>>>         There must be
> >> >>>>>>>>>>>>         > >>>> some sort of flaw in it.
> >> >>>>>>>>>>>>         > >>>> I have a suspicion where it is, I would just
> >> like
> >> >> to
> >> >>>>>>>>>>>>         confirm. The idea
> >> >>>>>>>>>>>>         > >>>> is bullet proof and it must be
> >> >>>>>>>>>>>>         > >>>> an implementation mess up. I would like to
> >> clarify
> >> >>>>> before
> >> >>>>>>>>>>>>         we draw a
> >> >>>>>>>>>>>>         > >>>> conclusion.
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>    Repartitioning the scattered events back
> to
> >> >> their
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> original
> >> >>>>>>>>>
> >> >>>>>>>>>>         > >>>>> partitions is the only way I know how to
> >> >> conclusively
> >> >>>>> deal
> >> >>>>>>>>>>>>         with
> >> >>>>>>>>>>>>         > >>>>> out-of-order events in a given time frame,
> >> and to
> >> >>>>> ensure
> >> >>>>>>>>>>>>         that the
> >> >>>>>>>>>>>>         > data
> >> >>>>>>>>>>>>         > >>>>> is
> >> >>>>>>>>>>>>         > >>>>> eventually consistent with the input
> events.
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>> If you have some code to share that
> >> illustrates
> >> >> your
> >> >>>>>>>>>>>>         approach, I
> >> >>>>>>>>>>>>         > would
> >> >>>>>>>>>>>>         > >>>>> be
> >> >>>>>>>>>>>>         > >>>>> very grateful as it would remove any
> >> >>>>> misunderstandings
> >> >>>>>>>>>>>>         that I may
> >> >>>>>>>>>>>>         > have.
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>> ah okay you were looking for my code. I
> don't
> >> have
> >> >>>>>>>>>>>>         something easily
> >> >>>>>>>>>>>>         > >>>> readable here as its bloated with
> OO-patterns.
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>> its anyhow trivial:
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>> @Override
> >> >>>>>>>>>>>>         > >>>>      public T apply(K aggKey, V value, T
> >> >> aggregate)
> >> >>>>>>>>>>>>         > >>>>      {
> >> >>>>>>>>>>>>         > >>>>          Map<U, V> currentStateAsMap =
> >> >>>>> asMap(aggregate);
> >> >>>>>>>>>>>> <<
> >> >>>>>>>>>>>>         imaginary
> >> >>>>>>>>>>>>         > >>>>          U toModifyKey =
> mapper.apply(value);
> >> >>>>>>>>>>>>         > >>>>              << this is the place where
> people
> >> >>>>> actually
> >> >>>>>>>>>>>>         gonna have
> >> >>>>>>>>>>>>         > issues
> >> >>>>>>>>>>>>         > >>>> and why you probably couldn't do it. we
> would
> >> need
> >> >>>>> to find
> >> >>>>>>>>>>>>         a solution
> >> >>>>>>>>>>>>         > here.
> >> >>>>>>>>>>>>         > >>>> I didn't realize that yet.
> >> >>>>>>>>>>>>         > >>>>              << we propagate the field in
> the
> >> >>>>> joiner, so
> >> >>>>>>>>>>>>         that we can
> >> >>>>>>>>>>>>         > pick
> >> >>>>>>>>>>>>         > >>>> it up in an aggregate. Probably you have not
> >> >> thought
> >> >>>>> of
> >> >>>>>>>>>>>>         this in your
> >> >>>>>>>>>>>>         > >>>> approach right?
> >> >>>>>>>>>>>>         > >>>>              << I am very open to find a
> >> generic
> >> >>>>> solution
> >> >>>>>>>>>>>>         here. In my
> >> >>>>>>>>>>>>         > >>>> honest opinion this is broken in
> >> >> KTableImpl.GroupBy
> >> >>>>> that
> >> >>>>>>>>>>>> it
> >> >>>>>>>>>>>>         looses
> >> >>>>>>>>>>>>         > the keys
> >> >>>>>>>>>>>>         > >>>> and only maintains the aggregate key.
> >> >>>>>>>>>>>>         > >>>>              << I abstracted it away back
> >> then way
> >> >>>>> before
> >> >>>>>>>>>>>> i
> >> >>>>>>>>>>>> was
> >> >>>>>>>>>>>>         > thinking
> >> >>>>>>>>>>>>         > >>>> of oneToMany join. That is why I didn't
> >> realize
> >> >> its
> >> >>>>>>>>>>>>         significance here.
> >> >>>>>>>>>>>>         > >>>>              << Opinions?
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>          for (V m : current)
> >> >>>>>>>>>>>>         > >>>>          {
> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >> >>>>>>>>>>>>         > >>>>          }
> >> >>>>>>>>>>>>         > >>>>          if (isAdder)
> >> >>>>>>>>>>>>         > >>>>          {
> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(toModifyKey, value);
> >> >>>>>>>>>>>>         > >>>>          }
> >> >>>>>>>>>>>>         > >>>>          else
> >> >>>>>>>>>>>>         > >>>>          {
> >> >>>>>>>>>>>>         > >>>> currentStateAsMap.remove(toModifyKey);
> >> >>>>>>>>>>>>         > >>>> if(currentStateAsMap.isEmpty()){
> >> >>>>>>>>>>>>         > >>>>                  return null;
> >> >>>>>>>>>>>>         > >>>>              }
> >> >>>>>>>>>>>>         > >>>>          }
> >> >>>>>>>>>>>>         > >>>>          retrun
> >> asAggregateType(currentStateAsMap)
> >> >>>>>>>>>>>>         > >>>>      }
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>>
> >> >>>>>>>>>>>>         > >>>> Thanks,
> >> >>>>>>>>>>>>         > >>>>> Adam
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan
> Filipiak
> >> <
> >> >>>>>>>>>>>>         > Jan.Filipiak@trivago.com <mailto:
> >> >> Jan.Filipiak@trivago.com
> >> >>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>> wrote:
> >> >>>>>>>>>>>>         > >>>>>
> >> >>>>>>>>>>>>         > >>>>> Thanks Adam for bringing Matthias to speed!
> >> >>>>>>>>>>>>         > >>>>>> about the differences. I think re-keying
> >> back
> >> >>>>> should be
> >> >>>>>>>>>>>>         optional at
> >> >>>>>>>>>>>>         > >>>>>> best.
> >> >>>>>>>>>>>>         > >>>>>> I would say we return a KScatteredTable
> with
> >> >>>>> reshuffle()
> >> >>>>>>>>>>>>         returning
> >> >>>>>>>>>>>>         > >>>>>> KTable<originalKey,Joined> to make the
> >> backwards
> >> >>>>>>>>>>>>         repartitioning
> >> >>>>>>>>>>>>         > >>>>>> optional.
> >> >>>>>>>>>>>>         > >>>>>> I am also in a big favour of doing the out
> >> of
> >> >> order
> >> >>>>>>>>>>>>         processing using
> >> >>>>>>>>>>>>         > >>>>>> group
> >> >>>>>>>>>>>>         > >>>>>> by instead high water mark tracking.
> >> >>>>>>>>>>>>         > >>>>>> Just because unbounded growth is just
> scary
> >> + It
> >> >>>>> saves
> >> >>>>>>>>>>>> us
> >> >>>>>>>>>>>>         the header
> >> >>>>>>>>>>>>         > >>>>>> stuff.
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>> I think the abstraction of always
> >> repartitioning
> >> >>>>> back is
> >> >>>>>>>>>>>>         just not so
> >> >>>>>>>>>>>>         > >>>>>> strong. Like the work has been done before
> >> we
> >> >>>>> partition
> >> >>>>>>>>>>>>         back and
> >> >>>>>>>>>>>>         > >>>>>> grouping
> >> >>>>>>>>>>>>         > >>>>>> by something else afterwards is really
> >> common.
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >> >>>>>>>>>>>>         > >>>>>>
> >> >>>>>>>>>>>>         > >>>>>> Hi Matthias
> >> >>>>>>>>>>>>         > >>>>>>> Thank you for your feedback, I do
> >> appreciate
> >> >> it!
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> While name spacing would be possible, it
> >> would
> >> >>>>> require
> >> >>>>>>>>>>>> to
> >> >>>>>>>>>>>>         > deserialize
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
> >> overhead.
> >> >> I
> >> >>>>> would
> >> >>>>>>>>>>>>         suggest to
> >> >>>>>>>>>>>>         > no
> >> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead.
> >> If
> >> >> this
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> becomes a
> >> >>>>>>>>>
> >> >>>>>>>>>>         > problem in
> >> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name
> spacing
> >> >> later
> >> >>>>> on.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Agreed. I will go with using a reserved
> >> string
> >> >>>>> and
> >> >>>>>>>>>>>>         document it.
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> My main concern about the design it the
> >> type of
> >> >>>>> the
> >> >>>>>>>>>>>>         result KTable:
> >> >>>>>>>>>>>>         > If
> >> >>>>>>>>>>>>         > >>>>>>> I
> >> >>>>>>>>>>>>         > >>>>>>> understood the proposal correctly,
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> In your example, you have table1 and
> table2
> >> >>>>> swapped.
> >> >>>>>>>>>>>>         Here is how it
> >> >>>>>>>>>>>>         > >>>>>>> works
> >> >>>>>>>>>>>>         > >>>>>>> currently:
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> 1) table1 has the records that contain
> the
> >> >>>>> foreign key
> >> >>>>>>>>>>>>         within their
> >> >>>>>>>>>>>>         > >>>>>>> value.
> >> >>>>>>>>>>>>         > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >> >>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >> >>>>>>>>>>>>         > >>>>>>> <c,(fk=B,bar=3)>
> >> >>>>>>>>>>>>         > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> 2) A Value mapper is required to extract
> >> the
> >> >>>>> foreign
> >> >>>>>>>>>>>> key.
> >> >>>>>>>>>>>>         > >>>>>>> table1 foreign key mapper: ( value =>
> >> value.fk
> >> >>>>>>>>>>>>         <http://value.fk> )
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> The mapper is applied to each element in
> >> >> table1,
> >> >>>>> and a
> >> >>>>>>>>>>>>         new combined
> >> >>>>>>>>>>>>         > >>>>>>> key is
> >> >>>>>>>>>>>>         > >>>>>>> made:
> >> >>>>>>>>>>>>         > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> >> >>>>>>>>>>>> (fk=A,bar=2)>,
> >> >>>>>>>>>>>>         <B-c,
> >> >>>>>>>>>>>>         > >>>>>>> (fk=B,bar=3)>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> 3) The rekeyed events are copartitioned
> >> with
> >> >>>>> table2:
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> a) Stream Thread with Partition 0:
> >> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
> >> <A-b,
> >> >>>>>>>>>>>>         (fk=A,bar=2)>
> >> >>>>>>>>>>>>         > >>>>>>> Table2: <A,X>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> b) Stream Thread with Partition 1:
> >> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >> >>>>>>>>>>>>         > >>>>>>> Table2: <B,Y>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> 4) From here, they can be joined together
> >> >> locally
> >> >>>>> by
> >> >>>>>>>>>>>>         applying the
> >> >>>>>>>>>>>>         > >>>>>>> joiner
> >> >>>>>>>>>>>>         > >>>>>>> function.
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> At this point, Jan's design and my design
> >> >>>>> deviate. My
> >> >>>>>>>>>>>>         design goes
> >> >>>>>>>>>>>>         > on
> >> >>>>>>>>>>>>         > >>>>>>> to
> >> >>>>>>>>>>>>         > >>>>>>> repartition the data post-join and
> resolve
> >> >>>>> out-of-order
> >> >>>>>>>>>>>>         arrival of
> >> >>>>>>>>>>>>         > >>>>>>> records,
> >> >>>>>>>>>>>>         > >>>>>>> finally returning the data keyed just the
> >> >>>>> original key.
> >> >>>>>>>>>>>>         I do not
> >> >>>>>>>>>>>>         > >>>>>>> expose
> >> >>>>>>>>>>>>         > >>>>>>> the
> >> >>>>>>>>>>>>         > >>>>>>> CombinedKey or any of the internals
> >> outside of
> >> >> the
> >> >>>>>>>>>>>>         joinOnForeignKey
> >> >>>>>>>>>>>>         > >>>>>>> function. This does make for larger
> >> footprint,
> >> >>>>> but it
> >> >>>>>>>>>>>>         removes all
> >> >>>>>>>>>>>>         > >>>>>>> agency
> >> >>>>>>>>>>>>         > >>>>>>> for resolving out-of-order arrivals and
> >> >> handling
> >> >>>>>>>>>>>>         CombinedKeys from
> >> >>>>>>>>>>>>         > the
> >> >>>>>>>>>>>>         > >>>>>>> user. I believe that this makes the
> >> function
> >> >> much
> >> >>>>>>>>>>>> easier
> >> >>>>>>>>>>>>         to use.
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> Let me know if this helps resolve your
> >> >> questions,
> >> >>>>> and
> >> >>>>>>>>>>>>         please feel
> >> >>>>>>>>>>>>         > >>>>>>> free to
> >> >>>>>>>>>>>>         > >>>>>>> add anything else on your mind.
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> Thanks again,
> >> >>>>>>>>>>>>         > >>>>>>> Adam
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias
> J.
> >> >> Sax <
> >> >>>>>>>>>>>>         > >>>>>>> matthias@confluent.io <mailto:
> >> >>>>> matthias@confluent.io>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> wrote:
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>> Hi,
> >> >>>>>>>>>>>>         > >>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> I am just catching up on this thread. I
> >> did
> >> >> not
> >> >>>>> read
> >> >>>>>>>>>>>>         everything so
> >> >>>>>>>>>>>>         > >>>>>>>> far,
> >> >>>>>>>>>>>>         > >>>>>>>> but want to share couple of initial
> >> thoughts:
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Headers: I think there is a fundamental
> >> >>>>> difference
> >> >>>>>>>>>>>>         between header
> >> >>>>>>>>>>>>         > >>>>>>>> usage
> >> >>>>>>>>>>>>         > >>>>>>>> in this KIP and KP-258. For 258, we add
> >> >> headers
> >> >>>>> to
> >> >>>>>>>>>>>>         changelog topic
> >> >>>>>>>>>>>>         > >>>>>>>> that
> >> >>>>>>>>>>>>         > >>>>>>>> are owned by Kafka Streams and nobody
> >> else is
> >> >>>>> supposed
> >> >>>>>>>>>>>>         to write
> >> >>>>>>>>>>>>         > into
> >> >>>>>>>>>>>>         > >>>>>>>> them. In fact, no user header are
> written
> >> into
> >> >>>>> the
> >> >>>>>>>>>>>>         changelog topic
> >> >>>>>>>>>>>>         > >>>>>>>> and
> >> >>>>>>>>>>>>         > >>>>>>>> thus, there are not conflicts.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Nevertheless, I don't see a big issue
> with
> >> >> using
> >> >>>>>>>>>>>>         headers within
> >> >>>>>>>>>>>>         > >>>>>>>> Streams.
> >> >>>>>>>>>>>>         > >>>>>>>> As long as we document it, we can have
> >> some
> >> >>>>> "reserved"
> >> >>>>>>>>>>>>         header keys
> >> >>>>>>>>>>>>         > >>>>>>>> and
> >> >>>>>>>>>>>>         > >>>>>>>> users are not allowed to use when
> >> processing
> >> >>>>> data with
> >> >>>>>>>>>>>>         Kafka
> >> >>>>>>>>>>>>         > Streams.
> >> >>>>>>>>>>>>         > >>>>>>>> IMHO, this should be ok.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> I think there is a safe way to avoid
> >> >> conflicts,
> >> >>>>> since
> >> >>>>>>>>>>>> these
> >> >>>>>>>>>>>>         > headers
> >> >>>>>>>>>>>>         > >>>>>>>> are
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>> only needed in internal topics (I
> think):
> >> >>>>>>>>>>>>         > >>>>>>>>> For internal and changelog topics, we
> can
> >> >>>>> namespace
> >> >>>>>>>>>>>>         all headers:
> >> >>>>>>>>>>>>         > >>>>>>>>> * user-defined headers are namespaced
> as
> >> >>>>> "external."
> >> >>>>>>>>>>>> +
> >> >>>>>>>>>>>>         headerKey
> >> >>>>>>>>>>>>         > >>>>>>>>> * internal headers are namespaced as
> >> >>>>> "internal." +
> >> >>>>>>>>>>>>         headerKey
> >> >>>>>>>>>>>>         > >>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>> While name spacing would be possible,
> it
> >> >> would
> >> >>>>>>>>>>>> require
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> to
> >> >>>>>>>>>
> >> >>>>>>>>>>         > >>>>>>>> deserialize
> >> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
> >> overhead.
> >> >> I
> >> >>>>> would
> >> >>>>>>>>>>>>         suggest to
> >> >>>>>>>>>>>>         > no
> >> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead.
> >> If
> >> >> this
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> becomes a
> >> >>>>>>>>>
> >> >>>>>>>>>>         > problem in
> >> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name
> spacing
> >> >> later
> >> >>>>> on.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> My main concern about the design it the
> >> type
> >> >> of
> >> >>>>> the
> >> >>>>>>>>>>>>         result KTable:
> >> >>>>>>>>>>>>         > >>>>>>>> If I
> >> >>>>>>>>>>>>         > >>>>>>>> understood the proposal correctly,
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V1> table1 = ...
> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K2,V2> table2 = ...
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V3> joinedTable =
> >> >>>>> table1.join(table2,...);
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> implies that the `joinedTable` has the
> >> same
> >> >> key
> >> >>>>> as the
> >> >>>>>>>>>>>>         left input
> >> >>>>>>>>>>>>         > >>>>>>>> table.
> >> >>>>>>>>>>>>         > >>>>>>>> IMHO, this does not work because if
> table2
> >> >>>>> contains
> >> >>>>>>>>>>>>         multiple rows
> >> >>>>>>>>>>>>         > >>>>>>>> that
> >> >>>>>>>>>>>>         > >>>>>>>> join with a record in table1 (what is
> the
> >> main
> >> >>>>> purpose
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> of
> >> >>>>>>>>>
> >> >>>>>>>>>> a
> >> >>>>>>>>>>>>         > foreign
> >> >>>>>>>>>>>>         > >>>>>>>> key
> >> >>>>>>>>>>>>         > >>>>>>>> join), the result table would only
> >> contain a
> >> >>>>> single
> >> >>>>>>>>>>>>         join result,
> >> >>>>>>>>>>>>         > but
> >> >>>>>>>>>>>>         > >>>>>>>> not
> >> >>>>>>>>>>>>         > >>>>>>>> multiple.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Example:
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> table1 input stream: <A,X>
> >> >>>>>>>>>>>>         > >>>>>>>> table2 input stream: <a,(A,1)>,
> <b,(A,2)>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> We use table2 value a foreign key to
> >> table1
> >> >> key
> >> >>>>> (ie,
> >> >>>>>>>>>>>>         "A" joins).
> >> >>>>>>>>>>>>         > If
> >> >>>>>>>>>>>>         > >>>>>>>> the
> >> >>>>>>>>>>>>         > >>>>>>>> result key is the same key as key of
> >> table1,
> >> >> this
> >> >>>>>>>>>>>>         implies that the
> >> >>>>>>>>>>>>         > >>>>>>>> result can either be <A, join(X,1)> or
> <A,
> >> >>>>> join(X,2)>
> >> >>>>>>>>>>>>         but not
> >> >>>>>>>>>>>>         > both.
> >> >>>>>>>>>>>>         > >>>>>>>> Because the share the same key, whatever
> >> >> result
> >> >>>>> record
> >> >>>>>>>>>>>>         we emit
> >> >>>>>>>>>>>>         > later,
> >> >>>>>>>>>>>>         > >>>>>>>> overwrite the previous result.
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> This is the reason why Jan originally
> >> proposed
> >> >>>>> to use
> >> >>>>>>>>>>>> a
> >> >>>>>>>>>>>>         > combination
> >> >>>>>>>>>>>>         > >>>>>>>> of
> >> >>>>>>>>>>>>         > >>>>>>>> both primary keys of the input tables as
> >> key
> >> >> of
> >> >>>>> the
> >> >>>>>>>>>>>>         output table.
> >> >>>>>>>>>>>>         > >>>>>>>> This
> >> >>>>>>>>>>>>         > >>>>>>>> makes the keys of the output table
> unique
> >> and
> >> >> we
> >> >>>>> can
> >> >>>>>>>>>>>>         store both in
> >> >>>>>>>>>>>>         > >>>>>>>> the
> >> >>>>>>>>>>>>         > >>>>>>>> output table:
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
> >> >>>>> join(X,2)>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Thoughts?
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> -Matthias
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >> >>>>>>>>>>>>         > >>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>> Just on remark here.
> >> >>>>>>>>>>>>         > >>>>>>>>> The high-watermark could be
> disregarded.
> >> The
> >> >>>>> decision
> >> >>>>>>>>>>>>         about the
> >> >>>>>>>>>>>>         > >>>>>>>>> forward
> >> >>>>>>>>>>>>         > >>>>>>>>> depends on the size of the aggregated
> >> map.
> >> >>>>>>>>>>>>         > >>>>>>>>> Only 1 element long maps would be
> >> unpacked
> >> >> and
> >> >>>>>>>>>>>>         forwarded. 0
> >> >>>>>>>>>>>>         > element
> >> >>>>>>>>>>>>         > >>>>>>>>> maps
> >> >>>>>>>>>>>>         > >>>>>>>>> would be published as delete. Any other
> >> count
> >> >>>>>>>>>>>>         > >>>>>>>>> of map entries is in "waiting for
> correct
> >> >>>>> deletes to
> >> >>>>>>>>>>>>         > arrive"-state.
> >> >>>>>>>>>>>>         > >>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
> >> wrote:
> >> >>>>>>>>>>>>         > >>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>> It does look like I could replace the
> >> second
> >> >>>>>>>>>>>>         repartition store
> >> >>>>>>>>>>>>         > and
> >> >>>>>>>>>>>>         > >>>>>>>>>> highwater store with a groupBy and
> >> reduce.
> >> >>>>> However,
> >> >>>>>>>>>>>>         it looks
> >> >>>>>>>>>>>>         > like
> >> >>>>>>>>>>>>         > >>>>>>>>>> I
> >> >>>>>>>>>>>>         > >>>>>>>>>> would
> >> >>>>>>>>>>>>         > >>>>>>>>>> still need to store the highwater
> value
> >> >> within
> >> >>>>> the
> >> >>>>>>>>>>>>         materialized
> >> >>>>>>>>>>>>         > >>>>>>>>>> store,
> >> >>>>>>>>>>>>         > >>>>>>>>>>
> >> >>>>>>>>>>>>         > >>>>>>>>>> to
> >> >>>>>>>>>>>>         > >>>>>>>>> compare the arrival of out-of-order
> >> records
> >> >>>>> (assuming
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>> my
> >> >>>>>>>>>
> >> >>>>>>>>>>         > >>>>>>>>> understanding
> >> >>>>>>>>>>>>         > >>>>>>>>> of
> >> >>>>>>>>>>>>         > >>>>>>>>> THIS is correct...). This in effect is
> >> the
> >> >> same
> >> >>>>> as
> >> >>>>>>>>>>>> the
> >> >>>>>>>>>>>>         design I
> >> >>>>>>>>>>>>         > have
> >> >>>>>>>>>>>>         > >>>>>>>>> now,
> >> >>>>>>>>>>>>         > >>>>>>>>> just with the two tables merged
> together.
> >> >>>>>>>>>>>>         > >>>>>>>>>
> >> >>>>>>>>>>>>         >
> >> >>>>>>>>>>>>         >
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> --
> >> >>>>>>>>>> -- Guozhang
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> --
> >> >>>>>>>>> -- Guozhang
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi Jan and Adam,

Wow, thanks for doing that test, Adam. Those results are encouraging.

Thanks for your performance experience as well, Jan. I agree that avoiding
unnecessary join outputs is especially important when the fan-out is so
high. I suppose this could also be built into the implementation we're
discussing, but it wouldn't have to be specified in the KIP (since it's an
API-transparent optimization).

As far as whether or not to re-repartition the data, I didn't bring it up
because it sounded like the two of you agreed to leave the KIP as-is,
despite the disagreement.

If you want my opinion, I feel like both approaches are reasonable.
It sounds like Jan values more the potential for developers to optimize
their topologies to re-use the intermediate nodes, whereas Adam places more
value on having a single operator that people can use without extra steps
at the end.

Personally, although I do find it exceptionally annoying when a framework
gets in my way when I'm trying to optimize something, it seems better to go
for a single operation.
* Encapsulating the internal transitions gives us significant latitude in
the implementation (for example, joining only at the end, not in the middle
to avoid extra data copying and out-of-order resolution; how we represent
the first repartition keys (combined keys vs. value vectors), etc.). If we
publish something like a KScatteredTable with the right-partitioned joined
data, then the API pretty much locks in the implementation as well.
* The API seems simpler to understand and use. I do mean "seems"; if anyone
wants to make the case that KScatteredTable is actually simpler, I think
hypothetical usage code would help. From a relational algebra perspective,
it seems like KTable.join(KTable) should produce a new KTable in all cases.
* That said, there might still be room in the API for a different operation
like what Jan has proposed to scatter a KTable, and then do things like
join, re-group, etc from there... I'm not sure; I haven't thought through
all the consequences yet.

This is all just my opinion after thinking over the discussion so far...
-John

On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare <ad...@gmail.com>
wrote:

> Updated the PR to take into account John's feedback.
>
> I did some preliminary testing for the performance of the prefixScan. I
> have attached the file, but I will also include the text in the body here
> for archival purposes (I am not sure what happens to attached files). I
> also updated the PR and the KIP accordingly.
>
> Summary: It scales exceptionally well for scanning large values of
> records. As Jan mentioned previously, the real issue would be more around
> processing the resulting records after obtaining them. For instance, it
> takes approximately ~80-120 mS to flush the buffer and a further ~35-85mS
> to scan 27.5M records, obtaining matches for 2.5M of them. Iterating
> through the records just to generate a simple count takes ~ 40 times longer
> than the flush + scan combined.
>
> ============================================================================================
> Setup:
>
> ============================================================================================
> Java 9 with default settings aside from a 512 MB heap (Xmx512m, Xms512m)
> CPU: i7 2.2 Ghz.
>
> Note: I am using a slightly-modified, directly-accessible Kafka Streams
> RocksDB
> implementation (RocksDB.java, basically just avoiding the
> ProcessorContext).
> There are no modifications to the default RocksDB values provided in the
> 2.1/trunk release.
>
>
> keysize = 128 bytes
> valsize = 512 bytes
>
> Step 1:
> Write X positive matching events: (key = prefix + left-padded
> auto-incrementing integer)
> Step 2:
> Write 10X negative matching events (key = left-padded auto-incrementing
> integer)
> Step 3:
> Perform flush
> Step 4:
> Perform prefixScan
> Step 5:
> Iterate through return Iterator and validate the count of expected events.
>
>
> ============================================================================================
> Results:
>
> ============================================================================================
> X = 1k (11k events total)
> Flush Time = 39 mS
> Scan Time = 7 mS
> 6.9 MB disk
>
> --------------------------------------------------------------------------------------------
> X = 10k (110k events total)
> Flush Time = 45 mS
> Scan Time = 8 mS
> 127 MB
>
> --------------------------------------------------------------------------------------------
> X = 100k (1.1M events total)
> Test1:
> Flush Time = 60 mS
> Scan Time = 12 mS
> 678 MB
>
> Test2:
> Flush Time = 45 mS
> Scan Time = 7 mS
> 576 MB
>
> --------------------------------------------------------------------------------------------
> X = 1MB (11M events total)
> Test1:
> Flush Time = 52 mS
> Scan Time = 19 mS
> 7.2 GB
>
> Test2:
> Flush Time = 84 mS
> Scan Time = 34 mS
> 9.1 GB
>
> --------------------------------------------------------------------------------------------
> X = 2.5M (27.5M events total)
> Test1:
> Flush Time = 82 mS
> Scan Time = 63 mS
> 17GB - 276 sst files
>
> Test2:
> Flush Time = 116 mS
> Scan Time = 35 mS
> 23GB - 361 sst files
>
> Test3:
> Flush Time = 103 mS
> Scan Time = 82 mS
> 19 GB - 300 sst files
>
> --------------------------------------------------------------------------------------------
>
> I had to limit my testing on my laptop to X = 2.5M events. I tried to go
> to X = 10M (110M events) but RocksDB was going into the 100GB+ range and my
> laptop ran out of disk. More extensive testing could be done but I suspect
> that it would be in line with what we're seeing in the results above.
>
>
>
>
>
>
> At this point in time, I think the only major discussion point is really
> around what Jan and I have disagreed on: repartitioning back + resolving
> potential out of order issues or leaving that up to the client to handle.
>
>
> Thanks folks,
>
> Adam
>
>
> On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>>
>>
>> On 29.11.2018 15:14, John Roesler wrote:
>> > Hi all,
>> >
>> > Sorry that this discussion petered out... I think the 2.1 release
>> caused an
>> > extended distraction that pushed it off everyone's radar (which was
>> > precisely Adam's concern). Personally, I've also had some extend
>> > distractions of my own that kept (and continue to keep) me preoccupied.
>> >
>> > However, calling for a vote did wake me up, so I guess Jan was on the
>> right
>> > track!
>> >
>> > I've gone back and reviewed the whole KIP document and the prior
>> > discussion, and I'd like to offer a few thoughts:
>> >
>> > API Thoughts:
>> >
>> > 1. If I read the KIP right, you are proposing a many-to-one join. Could
>> we
>> > consider naming it manyToOneJoin? Or, if you prefer, flip the design
>> around
>> > and make it a oneToManyJoin?
>> >
>> > The proposed name "joinOnForeignKey" disguises the join type, and it
>> seems
>> > like it might trick some people into using it for a one-to-one join.
>> This
>> > would work, of course, but it would be super inefficient compared to a
>> > simple rekey-and-join.
>> >
>> > 2. I might have missed it, but I don't think it's specified whether
>> it's an
>> > inner, outer, or left join. I'm guessing an outer join, as (neglecting
>> IQ),
>> > the rest can be achieved by filtering or by handling it in the
>> ValueJoiner.
>> >
>> > 3. The arg list to joinOnForeignKey doesn't look quite right.
>> > 3a. Regarding Serialized: There are a few different paradigms in play in
>> > the Streams API, so it's confusing, but instead of three Serialized
>> args, I
>> > think it would be better to have one that allows (optionally) setting
>> the 4
>> > incoming serdes. The result serde is defined by the Materialized. The
>> > incoming serdes can be optional because they might already be available
>> on
>> > the source KTables, or the default serdes from the config might be
>> > applicable.
>> >
>> > 3b. Is the StreamPartitioner necessary? The other joins don't allow
>> setting
>> > one, and it seems like it might actually be harmful, since the rekey
>> > operation needs to produce results that are co-partitioned with the
>> "other"
>> > KTable.
>> >
>> > 4. I'm fine with the "reserved word" header, but I didn't actually
>> follow
>> > what Matthias meant about namespacing requiring "deserializing" the
>> record
>> > header. The headers are already Strings, so I don't think that
>> > deserialization is required. If we applied the namespace at source nodes
>> > and stripped it at sink nodes, this would be practically no overhead.
>> The
>> > advantage of the namespace idea is that no public API change wrt headers
>> > needs to happen, and no restrictions need to be placed on users'
>> headers.
>> >
>> > (Although I'm wondering if we can get away without the header at all...
>> > stay tuned)
>> >
>> > 5. I also didn't follow the discussion about the HWM table growing
>> without
>> > bound. As I read it, the HWM table is effectively implementing OCC to
>> > resolve the problem you noted with disordering when the rekey is
>> > reversed... particularly notable when the FK changes. As such, it only
>> > needs to track the most recent "version" (the offset in the source
>> > partition) of each key. Therefore, it should have the same number of
>> keys
>> > as the source table at all times.
>> >
>> > I see that you are aware of KIP-258, which I think might be relevant in
>> a
>> > couple of ways. One: it's just about storing the timestamp in the state
>> > store, but the ultimate idea is to effectively use the timestamp as an
>> OCC
>> > "version" to drop disordered updates. You wouldn't want to use the
>> > timestamp for this operation, but if you were to use a similar
>> mechanism to
>> > store the source offset in the store alongside the re-keyed values, then
>> > you could avoid a separate table.
>> >
>> > 6. You and Jan have been thinking about this for a long time, so I've
>> > probably missed something here, but I'm wondering if we can avoid the
>> HWM
>> > tracking at all and resolve out-of-order during a final join instead...
>> >
>> > Let's say we're joining a left table (Integer K: Letter FK, (other
>> data))
>> > to a right table (Letter K: (some data)).
>> >
>> > Left table:
>> > 1: (A, xyz)
>> > 2: (B, asd)
>> >
>> > Right table:
>> > A: EntityA
>> > B: EntityB
>> >
>> > We could do a rekey as you proposed with a combined key, but not
>> > propagating the value at all..
>> > Rekey table:
>> > A-1: (dummy value)
>> > B-2: (dummy value)
>> >
>> > Which we then join with the right table to produce:
>> > A-1: EntityA
>> > B-2: EntityB
>> >
>> > Which gets rekeyed back:
>> > 1: A, EntityA
>> > 2: B, EntityB
>> >
>> > And finally we do the actual join:
>> > Result table:
>> > 1: ((A, xyz), EntityA)
>> > 2: ((B, asd), EntityB)
>> >
>> > The thing is that in that last join, we have the opportunity to compare
>> the
>> > current FK in the left table with the incoming PK of the right table. If
>> > they don't match, we just drop the event, since it must be outdated.
>> >
>>
>> > In your KIP, you gave an example in which (1: A, xyz) gets updated to
>> (1:
>> > B, xyz), ultimately yielding a conundrum about whether the final state
>> > should be (1: null) or (1: joined-on-B). With the algorithm above, you
>> > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
>> > EntityB)). It seems like this does give you enough information to make
>> the
>> > right choice, regardless of disordering.
>>
>> Will check Adams patch, but this should work. As mentioned often I am
>> not convinced on partitioning back for the user automatically. I think
>> this is the real performance eater ;)
>>
>> >
>> >
>> > 7. Last thought... I'm a little concerned about the performance of the
>> > range scans when records change in the right table. You've said that
>> you've
>> > been using the algorithm you presented in production for a while. Can
>> you
>> > give us a sense of the performance characteristics you've observed?
>> >
>>
>> Make it work, make it fast, make it beautiful. The topmost thing here is
>> / was correctness. In practice I do not measure the performance of the
>> range scan. Usual cases I run this with is emitting 500k - 1kk rows
>> on a left hand side change. The range scan is just the work you gotta
>> do, also when you pack your data into different formats, usually the
>> rocks performance is very tight to the size of the data and we can't
>> really change that. It is more important for users to prevent useless
>> updates to begin with. My left hand side is guarded to drop changes that
>> are not going to change my join output.
>>
>> usually it's:
>>
>> drop unused fields and then don't forward if old.equals(new)
>>
>> regarding to the performance of creating an iterator for smaller
>> fanouts, users can still just do a group by first then anyways.
>>
>>
>>
>> > I could only think of one alternative, but I'm not sure if it's better
>> or
>> > worse... If the first re-key only needs to preserve the original key,
>> as I
>> > proposed in #6, then we could store a vector of keys in the value:
>> >
>> > Left table:
>> > 1: A,...
>> > 2: B,...
>> > 3: A,...
>> >
>> > Gets re-keyed:
>> > A: [1, 3]
>> > B: [2]
>> >
>> > Then, the rhs part of the join would only need a regular single-key
>> lookup.
>> > Of course we have to deal with the problem of large values, as there's
>> no
>> > bound on the number of lhs records that can reference rhs records.
>> Offhand,
>> > I'd say we could page the values, so when one row is past the
>> threshold, we
>> > append the key for the next page. Then in most cases, it would be a
>> single
>> > key lookup, but for large fan-out updates, it would be one per (max
>> value
>> > size)/(avg lhs key size).
>> >
>> > This seems more complex, though... Plus, I think there's some extra
>> > tracking we'd need to do to know when to emit a retraction. For example,
>> > when record 1 is deleted, the re-key table would just have (A: [3]).
>> Some
>> > kind of tombstone is needed so that the join result for 1 can also be
>> > retracted.
>> >
>> > That's all!
>> >
>> > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
>> > discussion has been slow.
>> > -John
>> >
>> >
>> > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
>> > wrote:
>> >
>> >> Id say you can just call the vote.
>> >>
>> >> that happens all the time, and if something comes up, it just goes back
>> >> to discuss.
>> >>
>> >> would not expect to much attention with another another email in this
>> >> thread.
>> >>
>> >> best Jan
>> >>
>> >> On 09.10.2018 13:56, Adam Bellemare wrote:
>> >>> Hello Contributors
>> >>>
>> >>> I know that 2.1 is about to be released, but I do need to bump this to
>> >> keep
>> >>> visibility up. I am still intending to push this through once
>> contributor
>> >>> feedback is given.
>> >>>
>> >>> Main points that need addressing:
>> >>> 1) Any way (or benefit) in structuring the current singular graph node
>> >> into
>> >>> multiple nodes? It has a whopping 25 parameters right now. I am a bit
>> >> fuzzy
>> >>> on how the optimizations are supposed to work, so I would appreciate
>> any
>> >>> help on this aspect.
>> >>>
>> >>> 2) Overall strategy for joining + resolving. This thread has much
>> >> discourse
>> >>> between Jan and I between the current highwater mark proposal and a
>> >> groupBy
>> >>> + reduce proposal. I am of the opinion that we need to strictly handle
>> >> any
>> >>> chance of out-of-order data and leave none of it up to the consumer.
>> Any
>> >>> comments or suggestions here would also help.
>> >>>
>> >>> 3) Anything else that you see that would prevent this from moving to a
>> >> vote?
>> >>>
>> >>> Thanks
>> >>>
>> >>> Adam
>> >>>
>> >>>
>> >>>
>> >>>
>> >>>
>> >>>
>> >>>
>> >>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> >> adam.bellemare@gmail.com>
>> >>> wrote:
>> >>>
>> >>>> Hi Jan
>> >>>>
>> >>>> With the Stores.windowStoreBuilder and Stores.persistentWindowStore,
>> you
>> >>>> actually only need to specify the amount of segments you want and how
>> >> large
>> >>>> they are. To the best of my understanding, what happens is that the
>> >>>> segments are automatically rolled over as new data with new
>> timestamps
>> >> are
>> >>>> created. We use this exact functionality in some of the work done
>> >>>> internally at my company. For reference, this is the hopping windowed
>> >> store.
>> >>>>
>> >>>>
>> >>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>> >>>>
>> >>>> In the code that I have provided, there are going to be two 24h
>> >> segments.
>> >>>> When a record is put into the windowStore, it will be inserted at
>> time
>> >> T in
>> >>>> both segments. The two segments will always overlap by 12h. As time
>> >> goes on
>> >>>> and new records are added (say at time T+12h+), the oldest segment
>> will
>> >> be
>> >>>> automatically deleted and a new segment created. The records are by
>> >> default
>> >>>> inserted with the context.timestamp(), such that it is the record
>> time,
>> >> not
>> >>>> the clock time, which is used.
>> >>>>
>> >>>> To the best of my understanding, the timestamps are retained when
>> >>>> restoring from the changelog.
>> >>>>
>> >>>> Basically, this is heavy-handed way to deal with TTL at a
>> segment-level,
>> >>>> instead of at an individual record level.
>> >>>>
>> >>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
>> Jan.Filipiak@trivago.com>
>> >>>> wrote:
>> >>>>
>> >>>>> Will that work? I expected it to blow up with ClassCastException or
>> >>>>> similar.
>> >>>>>
>> >>>>> You either would have to specify the window you fetch/put or iterate
>> >>>>> across all windows the key was found in right?
>> >>>>>
>> >>>>> I just hope the window-store doesn't check stream-time under the
>> hoods
>> >>>>> that would be a questionable interface.
>> >>>>>
>> >>>>> If it does: did you see my comment on checking all the windows
>> earlier?
>> >>>>> that would be needed to actually give reasonable time gurantees.
>> >>>>>
>> >>>>> Best
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> >>>>>> Hi Jan
>> >>>>>>
>> >>>>>> Check for  " highwaterMat " in the PR. I only changed the state
>> store,
>> >>>>> not
>> >>>>>> the ProcessorSupplier.
>> >>>>>>
>> >>>>>> Thanks,
>> >>>>>> Adam
>> >>>>>>
>> >>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> >> Jan.Filipiak@trivago.com
>> >>>>>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>> >>>>>>>
>> >>>>>>>> @Guozhang
>> >>>>>>>>
>> >>>>>>>> Thanks for the information. This is indeed something that will be
>> >>>>>>>> extremely
>> >>>>>>>> useful for this KIP.
>> >>>>>>>>
>> >>>>>>>> @Jan
>> >>>>>>>> Thanks for your explanations. That being said, I will not be
>> moving
>> >>>>> ahead
>> >>>>>>>> with an implementation using reshuffle/groupBy solution as you
>> >>>>> propose.
>> >>>>>>>> That being said, if you wish to implement it yourself off of my
>> >>>>> current PR
>> >>>>>>>> and submit it as a competitive alternative, I would be more than
>> >>>>> happy to
>> >>>>>>>> help vet that as an alternate solution. As it stands right now,
>> I do
>> >>>>> not
>> >>>>>>>> really have more time to invest into alternatives without there
>> >> being
>> >>>>> a
>> >>>>>>>> strong indication from the binding voters which they would
>> prefer.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>> Hey, total no worries. I think I personally gave up on the streams
>> >> DSL
>> >>>>> for
>> >>>>>>> some time already, otherwise I would have pulled this KIP through
>> >>>>> already.
>> >>>>>>> I am currently reimplementing my own DSL based on PAPI.
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>> I will look at finishing up my PR with the windowed state store
>> in
>> >> the
>> >>>>>>>> next
>> >>>>>>>> week or so, exercising it via tests, and then I will come back
>> for
>> >>>>> final
>> >>>>>>>> discussions. In the meantime, I hope that any of the binding
>> voters
>> >>>>> could
>> >>>>>>>> take a look at the KIP in the wiki. I have updated it according
>> to
>> >> the
>> >>>>>>>> latest plan:
>> >>>>>>>>
>> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> >>>>>>>> Support+non-key+joining+in+KTable
>> >>>>>>>>
>> >>>>>>>> I have also updated the KIP PR to use a windowed store. This
>> could
>> >> be
>> >>>>>>>> replaced by the results of KIP-258 whenever they are completed.
>> >>>>>>>> https://github.com/apache/kafka/pull/5527
>> >>>>>>>>
>> >>>>>>>> Thanks,
>> >>>>>>>>
>> >>>>>>>> Adam
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated in
>> the
>> >>>>> PR?
>> >>>>>>> expected it to change to Windowed<K>,Long Missing something?
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
>> wangguoz@gmail.com>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link,
>> as it
>> >>>>> is
>> >>>>>>>>> for
>> >>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
>> >>>>> want to
>> >>>>>>>>> have "handling out-of-order data for source KTable" such that
>> >>>>> instead of
>> >>>>>>>>> blindly apply the updates to the materialized store, i.e.
>> following
>> >>>>>>>>> offset
>> >>>>>>>>> ordering, we will reject updates that are older than the current
>> >>>>> key's
>> >>>>>>>>> timestamps, i.e. following timestamp ordering.
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> Guozhang
>> >>>>>>>>>
>> >>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>> >> wangguoz@gmail.com>
>> >>>>>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>> Hello Adam,
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
>> >> high
>> >>>>>>>>>> watermark store, now altered to be replaced with a window
>> store),
>> >> I
>> >>>>>>>>>> think
>> >>>>>>>>>> another current on-going KIP may actually help:
>> >>>>>>>>>>
>> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> This is for adding the timestamp into a key-value store (i.e.
>> only
>> >>>>> for
>> >>>>>>>>>> non-windowed KTable), and then one of its usage, as described
>> in
>> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we
>> can
>> >>>>> then
>> >>>>>>>>>> "reject" updates from the source topics if its timestamp is
>> >> smaller
>> >>>>> than
>> >>>>>>>>>> the current key's latest update timestamp. I think it is very
>> >>>>> similar to
>> >>>>>>>>>> what you have in mind for high watermark based filtering, while
>> >> you
>> >>>>> only
>> >>>>>>>>>> need to make sure that the timestamps of the joining records
>> are
>> >>>>>>>>>>
>> >>>>>>>>> correctly
>> >>>>>>>>>
>> >>>>>>>>>> inherited though the whole topology to the final stage.
>> >>>>>>>>>>
>> >>>>>>>>>> Note that this KIP is for key-value store and hence
>> non-windowed
>> >>>>> KTables
>> >>>>>>>>>> only, but for windowed KTables we do not really have a good
>> >> support
>> >>>>> for
>> >>>>>>>>>> their joins anyways (
>> >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> >>>>>>>>>> I
>> >>>>>>>>>> think we can just consider non-windowed KTable-KTable non-key
>> >> joins
>> >>>>> for
>> >>>>>>>>>> now. In which case, KIP-258 should help.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Guozhang
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>> >>>>> Jan.Filipiak@trivago.com
>> >>>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> >>>>>>>>>>>
>> >>>>>>>>>>> Hi Guozhang
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Current highwater mark implementation would grow endlessly
>> based
>> >>>>> on
>> >>>>>>>>>>>> primary key of original event. It is a pair of (<this table
>> >>>>> primary
>> >>>>>>>>>>>>
>> >>>>>>>>>>> key>,
>> >>>>>>>>>
>> >>>>>>>>>> <highest offset seen for that key>). This is used to
>> differentiate
>> >>>>>>>>>>>>
>> >>>>>>>>>>> between
>> >>>>>>>>>
>> >>>>>>>>>> late arrivals and new updates. My newest proposal would be to
>> >>>>> replace
>> >>>>>>>>>>>>
>> >>>>>>>>>>> it
>> >>>>>>>>>
>> >>>>>>>>>> with a Windowed state store of Duration N. This would allow the
>> >> same
>> >>>>>>>>>>>> behaviour, but cap the size based on time. This should allow
>> for
>> >>>>> all
>> >>>>>>>>>>>> late-arriving events to be processed, and should be
>> customizable
>> >>>>> by
>> >>>>>>>>>>>> the
>> >>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10
>> minutes
>> >> of
>> >>>>>>>>>>>>
>> >>>>>>>>>>> window,
>> >>>>>>>>>
>> >>>>>>>>>> or perhaps 7 days...).
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Hi Adam, using time based retention can do the trick here.
>> Even
>> >>>>> if I
>> >>>>>>>>>>> would still like to see the automatic repartitioning optional
>> >>>>> since I
>> >>>>>>>>>>>
>> >>>>>>>>>> would
>> >>>>>>>>>
>> >>>>>>>>>> just reshuffle again. With windowed store I am a little bit
>> >>>>> sceptical
>> >>>>>>>>>>>
>> >>>>>>>>>> about
>> >>>>>>>>>
>> >>>>>>>>>> how to determine the window. So esentially one could run into
>> >>>>> problems
>> >>>>>>>>>>>
>> >>>>>>>>>> when
>> >>>>>>>>>
>> >>>>>>>>>> the rapid change happens near a window border. I will check you
>> >>>>>>>>>>> implementation in detail, if its problematic, we could still
>> >> check
>> >>>>>>>>>>> _all_
>> >>>>>>>>>>> windows on read with not to bad performance impact I guess.
>> Will
>> >>>>> let
>> >>>>>>>>>>> you
>> >>>>>>>>>>> know if the implementation would be correct as is. I wouldn't
>> not
>> >>>>> like
>> >>>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>
>> >>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>> >> timestamp(B).
>> >>>>> I
>> >>>>>>>>>>>
>> >>>>>>>>>> think
>> >>>>>>>>>
>> >>>>>>>>>> we can't expect that.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> @Jan
>> >>>>>>>>>>>> I believe I understand what you mean now - thanks for the
>> >>>>> diagram, it
>> >>>>>>>>>>>> did really help. You are correct that I do not have the
>> original
>> >>>>>>>>>>>>
>> >>>>>>>>>>> primary
>> >>>>>>>>>
>> >>>>>>>>>> key available, and I can see that if it was available then you
>> >>>>> would be
>> >>>>>>>>>>>> able to add and remove events from the Map. That being said,
>> I
>> >>>>>>>>>>>>
>> >>>>>>>>>>> encourage
>> >>>>>>>>>
>> >>>>>>>>>> you to finish your diagrams / charts just for clarity for
>> everyone
>> >>>>>>>>>>>>
>> >>>>>>>>>>> else.
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
>> >>>>> understand
>> >>>>>>>>>>>>
>> >>>>>>>>>>> the benefits for the rest. Sorry about the original primary
>> key,
>> >> We
>> >>>>>>>>>>> have
>> >>>>>>>>>>> join and Group by implemented our own in PAPI and basically
>> not
>> >>>>> using
>> >>>>>>>>>>>
>> >>>>>>>>>> any
>> >>>>>>>>>
>> >>>>>>>>>> DSL (Just the abstraction). Completely missed that in original
>> DSL
>> >>>>> its
>> >>>>>>>>>>>
>> >>>>>>>>>> not
>> >>>>>>>>>
>> >>>>>>>>>> there and just assumed it. total brain mess up on my end. Will
>> >>>>> finish
>> >>>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>
>> >>>>>>>>>> chart as soon as i get a quite evening this week.
>> >>>>>>>>>>>
>> >>>>>>>>>>> My follow up question for you is, won't the Map stay inside
>> the
>> >>>>> State
>> >>>>>>>>>>>
>> >>>>>>>>>>>> Store indefinitely after all of the changes have propagated?
>> >> Isn't
>> >>>>>>>>>>>> this
>> >>>>>>>>>>>> effectively the same as a highwater mark state store?
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna
>> return
>> >>>>> `null`
>> >>>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>
>> >>>>>>>>>> the key is removed from the keyspace. But there is going to be
>> a
>> >>>>> store
>> >>>>>>>>>>> 100%, the good thing is that I can use this store directly for
>> >>>>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
>> >>>>> satisfying
>> >>>>>>>>>>> all gurantees needed for further groupby / join. The Windowed
>> >>>>> store is
>> >>>>>>>>>>>
>> >>>>>>>>>> not
>> >>>>>>>>>
>> >>>>>>>>>> keeping the values, so for the next statefull operation we
>> would
>> >>>>>>>>>>> need to instantiate an extra store. or we have the window
>> store
>> >>>>> also
>> >>>>>>>>>>>
>> >>>>>>>>>> have
>> >>>>>>>>>
>> >>>>>>>>>> the values then.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Long story short. if we can flip in a custom group by before
>> >>>>>>>>>>> repartitioning to the original primary key i think it would
>> help
>> >>>>> the
>> >>>>>>>>>>>
>> >>>>>>>>>> users
>> >>>>>>>>>
>> >>>>>>>>>> big time in building efficient apps. Given the original primary
>> >> key
>> >>>>>>>>>>>
>> >>>>>>>>>> issue I
>> >>>>>>>>>
>> >>>>>>>>>> understand that we do not have a solid foundation to build on.
>> >>>>>>>>>>> Leaving primary key carry along to the user. very
>> unfortunate. I
>> >>>>> could
>> >>>>>>>>>>> understand the decision goes like that. I do not think its a
>> good
>> >>>>>>>>>>>
>> >>>>>>>>>> decision.
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Thanks
>> >>>>>>>>>>>> Adam
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>> >>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
>> dumbreprajakta311@gmail.com
>> >>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>>>         please remove me from this group
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> Jan.Filipiak@trivago.com
>> >>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         wrote:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > Hi Adam,
>> >>>>>>>>>>>>         >
>> >>>>>>>>>>>>         > give me some time, will make such a chart. last
>> time i
>> >>>>> didn't
>> >>>>>>>>>>>>         get along
>> >>>>>>>>>>>>         > well with giphy and ruined all your charts.
>> >>>>>>>>>>>>         > Hopefully i can get it done today
>> >>>>>>>>>>>>         >
>> >>>>>>>>>>>>         > On 08.09.2018 16:00, Adam Bellemare wrote:
>> >>>>>>>>>>>>         > > Hi Jan
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > > I have included a diagram of what I attempted on
>> the
>> >>>>> KIP.
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         >
>> >>>>>>>>>>>>
>> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> >>>>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> >>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> >>>>>>>>>>>>         <
>> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> >>>>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> >>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > > I attempted this back at the start of my own
>> >>>>> implementation
>> >>>>>>>>>>>> of
>> >>>>>>>>>>>>         this
>> >>>>>>>>>>>>         > > solution, and since I could not get it to work I
>> have
>> >>>>> since
>> >>>>>>>>>>>>         discarded the
>> >>>>>>>>>>>>         > > code. At this point in time, if you wish to
>> continue
>> >>>>> pursuing
>> >>>>>>>>>>>>         for your
>> >>>>>>>>>>>>         > > groupBy solution, I ask that you please create a
>> >>>>> diagram on
>> >>>>>>>>>>>>         the KIP
>> >>>>>>>>>>>>         > > carefully explaining your solution. Please feel
>> free
>> >> to
>> >>>>> use
>> >>>>>>>>>>>>         the image I
>> >>>>>>>>>>>>         > > just posted as a starting point. I am having
>> trouble
>> >>>>>>>>>>>>         understanding your
>> >>>>>>>>>>>>         > > explanations but I think that a carefully
>> constructed
>> >>>>> diagram
>> >>>>>>>>>>>>         will clear
>> >>>>>>>>>>>>         > up
>> >>>>>>>>>>>>         > > any misunderstandings. Alternately, please post a
>> >>>>>>>>>>>>         comprehensive PR with
>> >>>>>>>>>>>>         > > your solution. I can only guess at what you
>> mean, and
>> >>>>> since I
>> >>>>>>>>>>>>         value my
>> >>>>>>>>>>>>         > own
>> >>>>>>>>>>>>         > > time as much as you value yours, I believe it is
>> your
>> >>>>>>>>>>>>         responsibility to
>> >>>>>>>>>>>>         > > provide an implementation instead of me trying to
>> >> guess.
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > > Adam
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> Jan.Filipiak@trivago.com
>> >>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > > wrote:
>> >>>>>>>>>>>>         > >
>> >>>>>>>>>>>>         > >> Hi James,
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >> nice to see you beeing interested. kafka
>> streams at
>> >>>>> this
>> >>>>>>>>>>>>         point supports
>> >>>>>>>>>>>>         > >> all sorts of joins as long as both streams have
>> the
>> >>>>> same
>> >>>>>>>>>>>> key.
>> >>>>>>>>>>>>         > >> Adam is currently implementing a join where a
>> KTable
>> >>>>> and a
>> >>>>>>>>>>>>         KTable can
>> >>>>>>>>>>>>         > have
>> >>>>>>>>>>>>         > >> a one to many relation ship (1:n). We exploit
>> that
>> >>>>> rocksdb
>> >>>>>>>>>>>> is
>> >>>>>>>>>>>>
>> >>>>>>>>>>> a
>> >>>>>>>>>
>> >>>>>>>>>>         > >> datastore that keeps data sorted (At least
>> exposes an
>> >>>>> API to
>> >>>>>>>>>>>>         access the
>> >>>>>>>>>>>>         > >> stored data in a sorted fashion).
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >> I think the technical caveats are well
>> understood
>> >> now
>> >>>>> and we
>> >>>>>>>>>>>>
>> >>>>>>>>>>> are
>> >>>>>>>>>
>> >>>>>>>>>>         > basically
>> >>>>>>>>>>>>         > >> down to philosophy and API Design ( when Adam
>> sees
>> >> my
>> >>>>> newest
>> >>>>>>>>>>>>         message).
>> >>>>>>>>>>>>         > >> I have a lengthy track record of loosing those
>> kinda
>> >>>>>>>>>>>>         arguments within
>> >>>>>>>>>>>>         > the
>> >>>>>>>>>>>>         > >> streams community and I have no clue why. So I
>> >>>>> literally
>> >>>>>>>>>>>>         can't wait for
>> >>>>>>>>>>>>         > you
>> >>>>>>>>>>>>         > >> to churn through this thread and give you
>> opinion on
>> >>>>> how we
>> >>>>>>>>>>>>         should
>> >>>>>>>>>>>>         > design
>> >>>>>>>>>>>>         > >> the return type of the oneToManyJoin and how
>> many
>> >>>>> power we
>> >>>>>>>>>>>>         want to give
>> >>>>>>>>>>>>         > to
>> >>>>>>>>>>>>         > >> the user vs "simplicity" (where simplicity isn't
>> >>>>> really that
>> >>>>>>>>>>>>         as users
>> >>>>>>>>>>>>         > still
>> >>>>>>>>>>>>         > >> need to understand it I argue)
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >> waiting for you to join in on the discussion
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >> Best Jan
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >> On 07.09.2018 15:49, James Kwan wrote:
>> >>>>>>>>>>>>         > >>
>> >>>>>>>>>>>>         > >>> I am new to this group and I found this subject
>> >>>>>>>>>>>>         interesting.  Sounds
>> >>>>>>>>>>>>         > like
>> >>>>>>>>>>>>         > >>> you guys want to implement a join table of two
>> >>>>> streams? Is
>> >>>>>>>>>>>> there
>> >>>>>>>>>>>>         > somewhere
>> >>>>>>>>>>>>         > >>> I can see the original requirement or proposal?
>> >>>>>>>>>>>>         > >>>
>> >>>>>>>>>>>>         > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> >> Jan.Filipiak@trivago.com
>> >>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > >>>> wrote:
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>> I'm currently testing using a Windowed Store
>> to
>> >>>>> store the
>> >>>>>>>>>>>>         highwater
>> >>>>>>>>>>>>         > >>>>> mark.
>> >>>>>>>>>>>>         > >>>>> By all indications this should work fine,
>> with
>> >> the
>> >>>>> caveat
>> >>>>>>>>>>>>         being that
>> >>>>>>>>>>>>         > it
>> >>>>>>>>>>>>         > >>>>> can
>> >>>>>>>>>>>>         > >>>>> only resolve out-of-order arrival for up to
>> the
>> >>>>> size of
>> >>>>>>>>>>>>         the window
>> >>>>>>>>>>>>         > (ie:
>> >>>>>>>>>>>>         > >>>>> 24h, 72h, etc). This would remove the
>> possibility
>> >>>>> of it
>> >>>>>>>>>>>>
>> >>>>>>>>>>> being
>> >>>>>>>>>
>> >>>>>>>>>>         > unbounded
>> >>>>>>>>>>>>         > >>>>> in
>> >>>>>>>>>>>>         > >>>>> size.
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>> With regards to Jan's suggestion, I believe
>> this
>> >> is
>> >>>>> where
>> >>>>>>>>>>>>         we will
>> >>>>>>>>>>>>         > have
>> >>>>>>>>>>>>         > >>>>> to
>> >>>>>>>>>>>>         > >>>>> remain in disagreement. While I do not
>> disagree
>> >>>>> with your
>> >>>>>>>>>>>>         statement
>> >>>>>>>>>>>>         > >>>>> about
>> >>>>>>>>>>>>         > >>>>> there likely to be additional joins done in a
>> >>>>> real-world
>> >>>>>>>>>>>>         workflow, I
>> >>>>>>>>>>>>         > do
>> >>>>>>>>>>>>         > >>>>> not
>> >>>>>>>>>>>>         > >>>>> see how you can conclusively deal with
>> >> out-of-order
>> >>>>>>>>>>>> arrival
>> >>>>>>>>>>>> of
>> >>>>>>>>>>>>         > >>>>> foreign-key
>> >>>>>>>>>>>>         > >>>>> changes and subsequent joins. I have
>> attempted
>> >> what
>> >>>>> I
>> >>>>>>>>>>>>         think you have
>> >>>>>>>>>>>>         > >>>>> proposed (without a high-water, using
>> groupBy and
>> >>>>> reduce)
>> >>>>>>>>>>>>         and found
>> >>>>>>>>>>>>         > >>>>> that if
>> >>>>>>>>>>>>         > >>>>> the foreign key changes too quickly, or the
>> load
>> >> on
>> >>>>> a
>> >>>>>>>>>>>>         stream thread
>> >>>>>>>>>>>>         > is
>> >>>>>>>>>>>>         > >>>>> too
>> >>>>>>>>>>>>         > >>>>> high, the joined messages will arrive
>> >> out-of-order
>> >>>>> and be
>> >>>>>>>>>>>>         incorrectly
>> >>>>>>>>>>>>         > >>>>> propagated, such that an intermediate event
>> is
>> >>>>>>>>>>>> represented
>> >>>>>>>>>>>>         as the
>> >>>>>>>>>>>>         > final
>> >>>>>>>>>>>>         > >>>>> event.
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>> Can you shed some light on your groupBy
>> >>>>> implementation.
>> >>>>>>>>>>>>         There must be
>> >>>>>>>>>>>>         > >>>> some sort of flaw in it.
>> >>>>>>>>>>>>         > >>>> I have a suspicion where it is, I would just
>> like
>> >> to
>> >>>>>>>>>>>>         confirm. The idea
>> >>>>>>>>>>>>         > >>>> is bullet proof and it must be
>> >>>>>>>>>>>>         > >>>> an implementation mess up. I would like to
>> clarify
>> >>>>> before
>> >>>>>>>>>>>>         we draw a
>> >>>>>>>>>>>>         > >>>> conclusion.
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>    Repartitioning the scattered events back to
>> >> their
>> >>>>>>>>>>>>
>> >>>>>>>>>>> original
>> >>>>>>>>>
>> >>>>>>>>>>         > >>>>> partitions is the only way I know how to
>> >> conclusively
>> >>>>> deal
>> >>>>>>>>>>>>         with
>> >>>>>>>>>>>>         > >>>>> out-of-order events in a given time frame,
>> and to
>> >>>>> ensure
>> >>>>>>>>>>>>         that the
>> >>>>>>>>>>>>         > data
>> >>>>>>>>>>>>         > >>>>> is
>> >>>>>>>>>>>>         > >>>>> eventually consistent with the input events.
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>> If you have some code to share that
>> illustrates
>> >> your
>> >>>>>>>>>>>>         approach, I
>> >>>>>>>>>>>>         > would
>> >>>>>>>>>>>>         > >>>>> be
>> >>>>>>>>>>>>         > >>>>> very grateful as it would remove any
>> >>>>> misunderstandings
>> >>>>>>>>>>>>         that I may
>> >>>>>>>>>>>>         > have.
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>> ah okay you were looking for my code. I don't
>> have
>> >>>>>>>>>>>>         something easily
>> >>>>>>>>>>>>         > >>>> readable here as its bloated with OO-patterns.
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>> its anyhow trivial:
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>> @Override
>> >>>>>>>>>>>>         > >>>>      public T apply(K aggKey, V value, T
>> >> aggregate)
>> >>>>>>>>>>>>         > >>>>      {
>> >>>>>>>>>>>>         > >>>>          Map<U, V> currentStateAsMap =
>> >>>>> asMap(aggregate);
>> >>>>>>>>>>>> <<
>> >>>>>>>>>>>>         imaginary
>> >>>>>>>>>>>>         > >>>>          U toModifyKey = mapper.apply(value);
>> >>>>>>>>>>>>         > >>>>              << this is the place where people
>> >>>>> actually
>> >>>>>>>>>>>>         gonna have
>> >>>>>>>>>>>>         > issues
>> >>>>>>>>>>>>         > >>>> and why you probably couldn't do it. we would
>> need
>> >>>>> to find
>> >>>>>>>>>>>>         a solution
>> >>>>>>>>>>>>         > here.
>> >>>>>>>>>>>>         > >>>> I didn't realize that yet.
>> >>>>>>>>>>>>         > >>>>              << we propagate the field in the
>> >>>>> joiner, so
>> >>>>>>>>>>>>         that we can
>> >>>>>>>>>>>>         > pick
>> >>>>>>>>>>>>         > >>>> it up in an aggregate. Probably you have not
>> >> thought
>> >>>>> of
>> >>>>>>>>>>>>         this in your
>> >>>>>>>>>>>>         > >>>> approach right?
>> >>>>>>>>>>>>         > >>>>              << I am very open to find a
>> generic
>> >>>>> solution
>> >>>>>>>>>>>>         here. In my
>> >>>>>>>>>>>>         > >>>> honest opinion this is broken in
>> >> KTableImpl.GroupBy
>> >>>>> that
>> >>>>>>>>>>>> it
>> >>>>>>>>>>>>         looses
>> >>>>>>>>>>>>         > the keys
>> >>>>>>>>>>>>         > >>>> and only maintains the aggregate key.
>> >>>>>>>>>>>>         > >>>>              << I abstracted it away back
>> then way
>> >>>>> before
>> >>>>>>>>>>>> i
>> >>>>>>>>>>>> was
>> >>>>>>>>>>>>         > thinking
>> >>>>>>>>>>>>         > >>>> of oneToMany join. That is why I didn't
>> realize
>> >> its
>> >>>>>>>>>>>>         significance here.
>> >>>>>>>>>>>>         > >>>>              << Opinions?
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>          for (V m : current)
>> >>>>>>>>>>>>         > >>>>          {
>> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(mapper.apply(m), m);
>> >>>>>>>>>>>>         > >>>>          }
>> >>>>>>>>>>>>         > >>>>          if (isAdder)
>> >>>>>>>>>>>>         > >>>>          {
>> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(toModifyKey, value);
>> >>>>>>>>>>>>         > >>>>          }
>> >>>>>>>>>>>>         > >>>>          else
>> >>>>>>>>>>>>         > >>>>          {
>> >>>>>>>>>>>>         > >>>> currentStateAsMap.remove(toModifyKey);
>> >>>>>>>>>>>>         > >>>> if(currentStateAsMap.isEmpty()){
>> >>>>>>>>>>>>         > >>>>                  return null;
>> >>>>>>>>>>>>         > >>>>              }
>> >>>>>>>>>>>>         > >>>>          }
>> >>>>>>>>>>>>         > >>>>          retrun
>> asAggregateType(currentStateAsMap)
>> >>>>>>>>>>>>         > >>>>      }
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>>
>> >>>>>>>>>>>>         > >>>> Thanks,
>> >>>>>>>>>>>>         > >>>>> Adam
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak
>> <
>> >>>>>>>>>>>>         > Jan.Filipiak@trivago.com <mailto:
>> >> Jan.Filipiak@trivago.com
>> >>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > >>>>> wrote:
>> >>>>>>>>>>>>         > >>>>>
>> >>>>>>>>>>>>         > >>>>> Thanks Adam for bringing Matthias to speed!
>> >>>>>>>>>>>>         > >>>>>> about the differences. I think re-keying
>> back
>> >>>>> should be
>> >>>>>>>>>>>>         optional at
>> >>>>>>>>>>>>         > >>>>>> best.
>> >>>>>>>>>>>>         > >>>>>> I would say we return a KScatteredTable with
>> >>>>> reshuffle()
>> >>>>>>>>>>>>         returning
>> >>>>>>>>>>>>         > >>>>>> KTable<originalKey,Joined> to make the
>> backwards
>> >>>>>>>>>>>>         repartitioning
>> >>>>>>>>>>>>         > >>>>>> optional.
>> >>>>>>>>>>>>         > >>>>>> I am also in a big favour of doing the out
>> of
>> >> order
>> >>>>>>>>>>>>         processing using
>> >>>>>>>>>>>>         > >>>>>> group
>> >>>>>>>>>>>>         > >>>>>> by instead high water mark tracking.
>> >>>>>>>>>>>>         > >>>>>> Just because unbounded growth is just scary
>> + It
>> >>>>> saves
>> >>>>>>>>>>>> us
>> >>>>>>>>>>>>         the header
>> >>>>>>>>>>>>         > >>>>>> stuff.
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>> I think the abstraction of always
>> repartitioning
>> >>>>> back is
>> >>>>>>>>>>>>         just not so
>> >>>>>>>>>>>>         > >>>>>> strong. Like the work has been done before
>> we
>> >>>>> partition
>> >>>>>>>>>>>>         back and
>> >>>>>>>>>>>>         > >>>>>> grouping
>> >>>>>>>>>>>>         > >>>>>> by something else afterwards is really
>> common.
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>> >>>>>>>>>>>>         > >>>>>>
>> >>>>>>>>>>>>         > >>>>>> Hi Matthias
>> >>>>>>>>>>>>         > >>>>>>> Thank you for your feedback, I do
>> appreciate
>> >> it!
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> While name spacing would be possible, it
>> would
>> >>>>> require
>> >>>>>>>>>>>> to
>> >>>>>>>>>>>>         > deserialize
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
>> overhead.
>> >> I
>> >>>>> would
>> >>>>>>>>>>>>         suggest to
>> >>>>>>>>>>>>         > no
>> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead.
>> If
>> >> this
>> >>>>>>>>>>>>
>> >>>>>>>>>>> becomes a
>> >>>>>>>>>
>> >>>>>>>>>>         > problem in
>> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
>> >> later
>> >>>>> on.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Agreed. I will go with using a reserved
>> string
>> >>>>> and
>> >>>>>>>>>>>>         document it.
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> My main concern about the design it the
>> type of
>> >>>>> the
>> >>>>>>>>>>>>         result KTable:
>> >>>>>>>>>>>>         > If
>> >>>>>>>>>>>>         > >>>>>>> I
>> >>>>>>>>>>>>         > >>>>>>> understood the proposal correctly,
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> In your example, you have table1 and table2
>> >>>>> swapped.
>> >>>>>>>>>>>>         Here is how it
>> >>>>>>>>>>>>         > >>>>>>> works
>> >>>>>>>>>>>>         > >>>>>>> currently:
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> 1) table1 has the records that contain the
>> >>>>> foreign key
>> >>>>>>>>>>>>         within their
>> >>>>>>>>>>>>         > >>>>>>> value.
>> >>>>>>>>>>>>         > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>> >>>>>>>>>>>> <b,(fk=A,bar=2)>,
>> >>>>>>>>>>>>         > >>>>>>> <c,(fk=B,bar=3)>
>> >>>>>>>>>>>>         > >>>>>>> table2 input stream: <A,X>, <B,Y>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> 2) A Value mapper is required to extract
>> the
>> >>>>> foreign
>> >>>>>>>>>>>> key.
>> >>>>>>>>>>>>         > >>>>>>> table1 foreign key mapper: ( value =>
>> value.fk
>> >>>>>>>>>>>>         <http://value.fk> )
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> The mapper is applied to each element in
>> >> table1,
>> >>>>> and a
>> >>>>>>>>>>>>         new combined
>> >>>>>>>>>>>>         > >>>>>>> key is
>> >>>>>>>>>>>>         > >>>>>>> made:
>> >>>>>>>>>>>>         > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>> >>>>>>>>>>>> (fk=A,bar=2)>,
>> >>>>>>>>>>>>         <B-c,
>> >>>>>>>>>>>>         > >>>>>>> (fk=B,bar=3)>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> 3) The rekeyed events are copartitioned
>> with
>> >>>>> table2:
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> a) Stream Thread with Partition 0:
>> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
>> <A-b,
>> >>>>>>>>>>>>         (fk=A,bar=2)>
>> >>>>>>>>>>>>         > >>>>>>> Table2: <A,X>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> b) Stream Thread with Partition 1:
>> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>> >>>>>>>>>>>>         > >>>>>>> Table2: <B,Y>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> 4) From here, they can be joined together
>> >> locally
>> >>>>> by
>> >>>>>>>>>>>>         applying the
>> >>>>>>>>>>>>         > >>>>>>> joiner
>> >>>>>>>>>>>>         > >>>>>>> function.
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> At this point, Jan's design and my design
>> >>>>> deviate. My
>> >>>>>>>>>>>>         design goes
>> >>>>>>>>>>>>         > on
>> >>>>>>>>>>>>         > >>>>>>> to
>> >>>>>>>>>>>>         > >>>>>>> repartition the data post-join and resolve
>> >>>>> out-of-order
>> >>>>>>>>>>>>         arrival of
>> >>>>>>>>>>>>         > >>>>>>> records,
>> >>>>>>>>>>>>         > >>>>>>> finally returning the data keyed just the
>> >>>>> original key.
>> >>>>>>>>>>>>         I do not
>> >>>>>>>>>>>>         > >>>>>>> expose
>> >>>>>>>>>>>>         > >>>>>>> the
>> >>>>>>>>>>>>         > >>>>>>> CombinedKey or any of the internals
>> outside of
>> >> the
>> >>>>>>>>>>>>         joinOnForeignKey
>> >>>>>>>>>>>>         > >>>>>>> function. This does make for larger
>> footprint,
>> >>>>> but it
>> >>>>>>>>>>>>         removes all
>> >>>>>>>>>>>>         > >>>>>>> agency
>> >>>>>>>>>>>>         > >>>>>>> for resolving out-of-order arrivals and
>> >> handling
>> >>>>>>>>>>>>         CombinedKeys from
>> >>>>>>>>>>>>         > the
>> >>>>>>>>>>>>         > >>>>>>> user. I believe that this makes the
>> function
>> >> much
>> >>>>>>>>>>>> easier
>> >>>>>>>>>>>>         to use.
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> Let me know if this helps resolve your
>> >> questions,
>> >>>>> and
>> >>>>>>>>>>>>         please feel
>> >>>>>>>>>>>>         > >>>>>>> free to
>> >>>>>>>>>>>>         > >>>>>>> add anything else on your mind.
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> Thanks again,
>> >>>>>>>>>>>>         > >>>>>>> Adam
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
>> >> Sax <
>> >>>>>>>>>>>>         > >>>>>>> matthias@confluent.io <mailto:
>> >>>>> matthias@confluent.io>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> wrote:
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>> Hi,
>> >>>>>>>>>>>>         > >>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> I am just catching up on this thread. I
>> did
>> >> not
>> >>>>> read
>> >>>>>>>>>>>>         everything so
>> >>>>>>>>>>>>         > >>>>>>>> far,
>> >>>>>>>>>>>>         > >>>>>>>> but want to share couple of initial
>> thoughts:
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Headers: I think there is a fundamental
>> >>>>> difference
>> >>>>>>>>>>>>         between header
>> >>>>>>>>>>>>         > >>>>>>>> usage
>> >>>>>>>>>>>>         > >>>>>>>> in this KIP and KP-258. For 258, we add
>> >> headers
>> >>>>> to
>> >>>>>>>>>>>>         changelog topic
>> >>>>>>>>>>>>         > >>>>>>>> that
>> >>>>>>>>>>>>         > >>>>>>>> are owned by Kafka Streams and nobody
>> else is
>> >>>>> supposed
>> >>>>>>>>>>>>         to write
>> >>>>>>>>>>>>         > into
>> >>>>>>>>>>>>         > >>>>>>>> them. In fact, no user header are written
>> into
>> >>>>> the
>> >>>>>>>>>>>>         changelog topic
>> >>>>>>>>>>>>         > >>>>>>>> and
>> >>>>>>>>>>>>         > >>>>>>>> thus, there are not conflicts.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Nevertheless, I don't see a big issue with
>> >> using
>> >>>>>>>>>>>>         headers within
>> >>>>>>>>>>>>         > >>>>>>>> Streams.
>> >>>>>>>>>>>>         > >>>>>>>> As long as we document it, we can have
>> some
>> >>>>> "reserved"
>> >>>>>>>>>>>>         header keys
>> >>>>>>>>>>>>         > >>>>>>>> and
>> >>>>>>>>>>>>         > >>>>>>>> users are not allowed to use when
>> processing
>> >>>>> data with
>> >>>>>>>>>>>>         Kafka
>> >>>>>>>>>>>>         > Streams.
>> >>>>>>>>>>>>         > >>>>>>>> IMHO, this should be ok.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> I think there is a safe way to avoid
>> >> conflicts,
>> >>>>> since
>> >>>>>>>>>>>> these
>> >>>>>>>>>>>>         > headers
>> >>>>>>>>>>>>         > >>>>>>>> are
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>> only needed in internal topics (I think):
>> >>>>>>>>>>>>         > >>>>>>>>> For internal and changelog topics, we can
>> >>>>> namespace
>> >>>>>>>>>>>>         all headers:
>> >>>>>>>>>>>>         > >>>>>>>>> * user-defined headers are namespaced as
>> >>>>> "external."
>> >>>>>>>>>>>> +
>> >>>>>>>>>>>>         headerKey
>> >>>>>>>>>>>>         > >>>>>>>>> * internal headers are namespaced as
>> >>>>> "internal." +
>> >>>>>>>>>>>>         headerKey
>> >>>>>>>>>>>>         > >>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>> While name spacing would be possible, it
>> >> would
>> >>>>>>>>>>>> require
>> >>>>>>>>>>>>
>> >>>>>>>>>>> to
>> >>>>>>>>>
>> >>>>>>>>>>         > >>>>>>>> deserialize
>> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
>> overhead.
>> >> I
>> >>>>> would
>> >>>>>>>>>>>>         suggest to
>> >>>>>>>>>>>>         > no
>> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead.
>> If
>> >> this
>> >>>>>>>>>>>>
>> >>>>>>>>>>> becomes a
>> >>>>>>>>>
>> >>>>>>>>>>         > problem in
>> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
>> >> later
>> >>>>> on.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> My main concern about the design it the
>> type
>> >> of
>> >>>>> the
>> >>>>>>>>>>>>         result KTable:
>> >>>>>>>>>>>>         > >>>>>>>> If I
>> >>>>>>>>>>>>         > >>>>>>>> understood the proposal correctly,
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V1> table1 = ...
>> >>>>>>>>>>>>         > >>>>>>>> KTable<K2,V2> table2 = ...
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V3> joinedTable =
>> >>>>> table1.join(table2,...);
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> implies that the `joinedTable` has the
>> same
>> >> key
>> >>>>> as the
>> >>>>>>>>>>>>         left input
>> >>>>>>>>>>>>         > >>>>>>>> table.
>> >>>>>>>>>>>>         > >>>>>>>> IMHO, this does not work because if table2
>> >>>>> contains
>> >>>>>>>>>>>>         multiple rows
>> >>>>>>>>>>>>         > >>>>>>>> that
>> >>>>>>>>>>>>         > >>>>>>>> join with a record in table1 (what is the
>> main
>> >>>>> purpose
>> >>>>>>>>>>>>
>> >>>>>>>>>>> of
>> >>>>>>>>>
>> >>>>>>>>>> a
>> >>>>>>>>>>>>         > foreign
>> >>>>>>>>>>>>         > >>>>>>>> key
>> >>>>>>>>>>>>         > >>>>>>>> join), the result table would only
>> contain a
>> >>>>> single
>> >>>>>>>>>>>>         join result,
>> >>>>>>>>>>>>         > but
>> >>>>>>>>>>>>         > >>>>>>>> not
>> >>>>>>>>>>>>         > >>>>>>>> multiple.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Example:
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> table1 input stream: <A,X>
>> >>>>>>>>>>>>         > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> We use table2 value a foreign key to
>> table1
>> >> key
>> >>>>> (ie,
>> >>>>>>>>>>>>         "A" joins).
>> >>>>>>>>>>>>         > If
>> >>>>>>>>>>>>         > >>>>>>>> the
>> >>>>>>>>>>>>         > >>>>>>>> result key is the same key as key of
>> table1,
>> >> this
>> >>>>>>>>>>>>         implies that the
>> >>>>>>>>>>>>         > >>>>>>>> result can either be <A, join(X,1)> or <A,
>> >>>>> join(X,2)>
>> >>>>>>>>>>>>         but not
>> >>>>>>>>>>>>         > both.
>> >>>>>>>>>>>>         > >>>>>>>> Because the share the same key, whatever
>> >> result
>> >>>>> record
>> >>>>>>>>>>>>         we emit
>> >>>>>>>>>>>>         > later,
>> >>>>>>>>>>>>         > >>>>>>>> overwrite the previous result.
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> This is the reason why Jan originally
>> proposed
>> >>>>> to use
>> >>>>>>>>>>>> a
>> >>>>>>>>>>>>         > combination
>> >>>>>>>>>>>>         > >>>>>>>> of
>> >>>>>>>>>>>>         > >>>>>>>> both primary keys of the input tables as
>> key
>> >> of
>> >>>>> the
>> >>>>>>>>>>>>         output table.
>> >>>>>>>>>>>>         > >>>>>>>> This
>> >>>>>>>>>>>>         > >>>>>>>> makes the keys of the output table unique
>> and
>> >> we
>> >>>>> can
>> >>>>>>>>>>>>         store both in
>> >>>>>>>>>>>>         > >>>>>>>> the
>> >>>>>>>>>>>>         > >>>>>>>> output table:
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>> >>>>> join(X,2)>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Thoughts?
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> -Matthias
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>> >>>>>>>>>>>>         > >>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>> Just on remark here.
>> >>>>>>>>>>>>         > >>>>>>>>> The high-watermark could be disregarded.
>> The
>> >>>>> decision
>> >>>>>>>>>>>>         about the
>> >>>>>>>>>>>>         > >>>>>>>>> forward
>> >>>>>>>>>>>>         > >>>>>>>>> depends on the size of the aggregated
>> map.
>> >>>>>>>>>>>>         > >>>>>>>>> Only 1 element long maps would be
>> unpacked
>> >> and
>> >>>>>>>>>>>>         forwarded. 0
>> >>>>>>>>>>>>         > element
>> >>>>>>>>>>>>         > >>>>>>>>> maps
>> >>>>>>>>>>>>         > >>>>>>>>> would be published as delete. Any other
>> count
>> >>>>>>>>>>>>         > >>>>>>>>> of map entries is in "waiting for correct
>> >>>>> deletes to
>> >>>>>>>>>>>>         > arrive"-state.
>> >>>>>>>>>>>>         > >>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare
>> wrote:
>> >>>>>>>>>>>>         > >>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>> It does look like I could replace the
>> second
>> >>>>>>>>>>>>         repartition store
>> >>>>>>>>>>>>         > and
>> >>>>>>>>>>>>         > >>>>>>>>>> highwater store with a groupBy and
>> reduce.
>> >>>>> However,
>> >>>>>>>>>>>>         it looks
>> >>>>>>>>>>>>         > like
>> >>>>>>>>>>>>         > >>>>>>>>>> I
>> >>>>>>>>>>>>         > >>>>>>>>>> would
>> >>>>>>>>>>>>         > >>>>>>>>>> still need to store the highwater value
>> >> within
>> >>>>> the
>> >>>>>>>>>>>>         materialized
>> >>>>>>>>>>>>         > >>>>>>>>>> store,
>> >>>>>>>>>>>>         > >>>>>>>>>>
>> >>>>>>>>>>>>         > >>>>>>>>>> to
>> >>>>>>>>>>>>         > >>>>>>>>> compare the arrival of out-of-order
>> records
>> >>>>> (assuming
>> >>>>>>>>>>>>
>> >>>>>>>>>>> my
>> >>>>>>>>>
>> >>>>>>>>>>         > >>>>>>>>> understanding
>> >>>>>>>>>>>>         > >>>>>>>>> of
>> >>>>>>>>>>>>         > >>>>>>>>> THIS is correct...). This in effect is
>> the
>> >> same
>> >>>>> as
>> >>>>>>>>>>>> the
>> >>>>>>>>>>>>         design I
>> >>>>>>>>>>>>         > have
>> >>>>>>>>>>>>         > >>>>>>>>> now,
>> >>>>>>>>>>>>         > >>>>>>>>> just with the two tables merged together.
>> >>>>>>>>>>>>         > >>>>>>>>>
>> >>>>>>>>>>>>         >
>> >>>>>>>>>>>>         >
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> --
>> >>>>>>>>>> -- Guozhang
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> --
>> >>>>>>>>> -- Guozhang
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Updated the PR to take into account John's feedback.

I did some preliminary testing for the performance of the prefixScan. I
have attached the file, but I will also include the text in the body here
for archival purposes (I am not sure what happens to attached files). I
also updated the PR and the KIP accordingly.

Summary: It scales exceptionally well for scanning large values of records.
As Jan mentioned previously, the real issue would be more around processing
the resulting records after obtaining them. For instance, it takes
approximately ~80-120 mS to flush the buffer and a further ~35-85mS to scan
27.5M records, obtaining matches for 2.5M of them. Iterating through the
records just to generate a simple count takes ~ 40 times longer than the
flush + scan combined.
============================================================================================
Setup:
============================================================================================
Java 9 with default settings aside from a 512 MB heap (Xmx512m, Xms512m)
CPU: i7 2.2 Ghz.

Note: I am using a slightly-modified, directly-accessible Kafka Streams
RocksDB
implementation (RocksDB.java, basically just avoiding the
ProcessorContext).
There are no modifications to the default RocksDB values provided in the
2.1/trunk release.


keysize = 128 bytes
valsize = 512 bytes

Step 1:
Write X positive matching events: (key = prefix + left-padded
auto-incrementing integer)
Step 2:
Write 10X negative matching events (key = left-padded auto-incrementing
integer)
Step 3:
Perform flush
Step 4:
Perform prefixScan
Step 5:
Iterate through return Iterator and validate the count of expected events.

============================================================================================
Results:
============================================================================================
X = 1k (11k events total)
Flush Time = 39 mS
Scan Time = 7 mS
6.9 MB disk
--------------------------------------------------------------------------------------------
X = 10k (110k events total)
Flush Time = 45 mS
Scan Time = 8 mS
127 MB
--------------------------------------------------------------------------------------------
X = 100k (1.1M events total)
Test1:
Flush Time = 60 mS
Scan Time = 12 mS
678 MB

Test2:
Flush Time = 45 mS
Scan Time = 7 mS
576 MB
--------------------------------------------------------------------------------------------
X = 1MB (11M events total)
Test1:
Flush Time = 52 mS
Scan Time = 19 mS
7.2 GB

Test2:
Flush Time = 84 mS
Scan Time = 34 mS
9.1 GB
--------------------------------------------------------------------------------------------
X = 2.5M (27.5M events total)
Test1:
Flush Time = 82 mS
Scan Time = 63 mS
17GB - 276 sst files

Test2:
Flush Time = 116 mS
Scan Time = 35 mS
23GB - 361 sst files

Test3:
Flush Time = 103 mS
Scan Time = 82 mS
19 GB - 300 sst files
--------------------------------------------------------------------------------------------

I had to limit my testing on my laptop to X = 2.5M events. I tried to go to
X = 10M (110M events) but RocksDB was going into the 100GB+ range and my
laptop ran out of disk. More extensive testing could be done but I suspect
that it would be in line with what we're seeing in the results above.






At this point in time, I think the only major discussion point is really
around what Jan and I have disagreed on: repartitioning back + resolving
potential out of order issues or leaving that up to the client to handle.


Thanks folks,

Adam


On Mon, Dec 3, 2018 at 4:34 AM Jan Filipiak <Ja...@trivago.com>
wrote:

>
>
> On 29.11.2018 15:14, John Roesler wrote:
> > Hi all,
> >
> > Sorry that this discussion petered out... I think the 2.1 release caused
> an
> > extended distraction that pushed it off everyone's radar (which was
> > precisely Adam's concern). Personally, I've also had some extend
> > distractions of my own that kept (and continue to keep) me preoccupied.
> >
> > However, calling for a vote did wake me up, so I guess Jan was on the
> right
> > track!
> >
> > I've gone back and reviewed the whole KIP document and the prior
> > discussion, and I'd like to offer a few thoughts:
> >
> > API Thoughts:
> >
> > 1. If I read the KIP right, you are proposing a many-to-one join. Could
> we
> > consider naming it manyToOneJoin? Or, if you prefer, flip the design
> around
> > and make it a oneToManyJoin?
> >
> > The proposed name "joinOnForeignKey" disguises the join type, and it
> seems
> > like it might trick some people into using it for a one-to-one join. This
> > would work, of course, but it would be super inefficient compared to a
> > simple rekey-and-join.
> >
> > 2. I might have missed it, but I don't think it's specified whether it's
> an
> > inner, outer, or left join. I'm guessing an outer join, as (neglecting
> IQ),
> > the rest can be achieved by filtering or by handling it in the
> ValueJoiner.
> >
> > 3. The arg list to joinOnForeignKey doesn't look quite right.
> > 3a. Regarding Serialized: There are a few different paradigms in play in
> > the Streams API, so it's confusing, but instead of three Serialized
> args, I
> > think it would be better to have one that allows (optionally) setting
> the 4
> > incoming serdes. The result serde is defined by the Materialized. The
> > incoming serdes can be optional because they might already be available
> on
> > the source KTables, or the default serdes from the config might be
> > applicable.
> >
> > 3b. Is the StreamPartitioner necessary? The other joins don't allow
> setting
> > one, and it seems like it might actually be harmful, since the rekey
> > operation needs to produce results that are co-partitioned with the
> "other"
> > KTable.
> >
> > 4. I'm fine with the "reserved word" header, but I didn't actually follow
> > what Matthias meant about namespacing requiring "deserializing" the
> record
> > header. The headers are already Strings, so I don't think that
> > deserialization is required. If we applied the namespace at source nodes
> > and stripped it at sink nodes, this would be practically no overhead. The
> > advantage of the namespace idea is that no public API change wrt headers
> > needs to happen, and no restrictions need to be placed on users' headers.
> >
> > (Although I'm wondering if we can get away without the header at all...
> > stay tuned)
> >
> > 5. I also didn't follow the discussion about the HWM table growing
> without
> > bound. As I read it, the HWM table is effectively implementing OCC to
> > resolve the problem you noted with disordering when the rekey is
> > reversed... particularly notable when the FK changes. As such, it only
> > needs to track the most recent "version" (the offset in the source
> > partition) of each key. Therefore, it should have the same number of keys
> > as the source table at all times.
> >
> > I see that you are aware of KIP-258, which I think might be relevant in a
> > couple of ways. One: it's just about storing the timestamp in the state
> > store, but the ultimate idea is to effectively use the timestamp as an
> OCC
> > "version" to drop disordered updates. You wouldn't want to use the
> > timestamp for this operation, but if you were to use a similar mechanism
> to
> > store the source offset in the store alongside the re-keyed values, then
> > you could avoid a separate table.
> >
> > 6. You and Jan have been thinking about this for a long time, so I've
> > probably missed something here, but I'm wondering if we can avoid the HWM
> > tracking at all and resolve out-of-order during a final join instead...
> >
> > Let's say we're joining a left table (Integer K: Letter FK, (other data))
> > to a right table (Letter K: (some data)).
> >
> > Left table:
> > 1: (A, xyz)
> > 2: (B, asd)
> >
> > Right table:
> > A: EntityA
> > B: EntityB
> >
> > We could do a rekey as you proposed with a combined key, but not
> > propagating the value at all..
> > Rekey table:
> > A-1: (dummy value)
> > B-2: (dummy value)
> >
> > Which we then join with the right table to produce:
> > A-1: EntityA
> > B-2: EntityB
> >
> > Which gets rekeyed back:
> > 1: A, EntityA
> > 2: B, EntityB
> >
> > And finally we do the actual join:
> > Result table:
> > 1: ((A, xyz), EntityA)
> > 2: ((B, asd), EntityB)
> >
> > The thing is that in that last join, we have the opportunity to compare
> the
> > current FK in the left table with the incoming PK of the right table. If
> > they don't match, we just drop the event, since it must be outdated.
> >
>
> > In your KIP, you gave an example in which (1: A, xyz) gets updated to (1:
> > B, xyz), ultimately yielding a conundrum about whether the final state
> > should be (1: null) or (1: joined-on-B). With the algorithm above, you
> > would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> > EntityB)). It seems like this does give you enough information to make
> the
> > right choice, regardless of disordering.
>
> Will check Adams patch, but this should work. As mentioned often I am
> not convinced on partitioning back for the user automatically. I think
> this is the real performance eater ;)
>
> >
> >
> > 7. Last thought... I'm a little concerned about the performance of the
> > range scans when records change in the right table. You've said that
> you've
> > been using the algorithm you presented in production for a while. Can you
> > give us a sense of the performance characteristics you've observed?
> >
>
> Make it work, make it fast, make it beautiful. The topmost thing here is
> / was correctness. In practice I do not measure the performance of the
> range scan. Usual cases I run this with is emitting 500k - 1kk rows
> on a left hand side change. The range scan is just the work you gotta
> do, also when you pack your data into different formats, usually the
> rocks performance is very tight to the size of the data and we can't
> really change that. It is more important for users to prevent useless
> updates to begin with. My left hand side is guarded to drop changes that
> are not going to change my join output.
>
> usually it's:
>
> drop unused fields and then don't forward if old.equals(new)
>
> regarding to the performance of creating an iterator for smaller
> fanouts, users can still just do a group by first then anyways.
>
>
>
> > I could only think of one alternative, but I'm not sure if it's better or
> > worse... If the first re-key only needs to preserve the original key, as
> I
> > proposed in #6, then we could store a vector of keys in the value:
> >
> > Left table:
> > 1: A,...
> > 2: B,...
> > 3: A,...
> >
> > Gets re-keyed:
> > A: [1, 3]
> > B: [2]
> >
> > Then, the rhs part of the join would only need a regular single-key
> lookup.
> > Of course we have to deal with the problem of large values, as there's no
> > bound on the number of lhs records that can reference rhs records.
> Offhand,
> > I'd say we could page the values, so when one row is past the threshold,
> we
> > append the key for the next page. Then in most cases, it would be a
> single
> > key lookup, but for large fan-out updates, it would be one per (max value
> > size)/(avg lhs key size).
> >
> > This seems more complex, though... Plus, I think there's some extra
> > tracking we'd need to do to know when to emit a retraction. For example,
> > when record 1 is deleted, the re-key table would just have (A: [3]). Some
> > kind of tombstone is needed so that the join result for 1 can also be
> > retracted.
> >
> > That's all!
> >
> > Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> > discussion has been slow.
> > -John
> >
> >
> > On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >> Id say you can just call the vote.
> >>
> >> that happens all the time, and if something comes up, it just goes back
> >> to discuss.
> >>
> >> would not expect to much attention with another another email in this
> >> thread.
> >>
> >> best Jan
> >>
> >> On 09.10.2018 13:56, Adam Bellemare wrote:
> >>> Hello Contributors
> >>>
> >>> I know that 2.1 is about to be released, but I do need to bump this to
> >> keep
> >>> visibility up. I am still intending to push this through once
> contributor
> >>> feedback is given.
> >>>
> >>> Main points that need addressing:
> >>> 1) Any way (or benefit) in structuring the current singular graph node
> >> into
> >>> multiple nodes? It has a whopping 25 parameters right now. I am a bit
> >> fuzzy
> >>> on how the optimizations are supposed to work, so I would appreciate
> any
> >>> help on this aspect.
> >>>
> >>> 2) Overall strategy for joining + resolving. This thread has much
> >> discourse
> >>> between Jan and I between the current highwater mark proposal and a
> >> groupBy
> >>> + reduce proposal. I am of the opinion that we need to strictly handle
> >> any
> >>> chance of out-of-order data and leave none of it up to the consumer.
> Any
> >>> comments or suggestions here would also help.
> >>>
> >>> 3) Anything else that you see that would prevent this from moving to a
> >> vote?
> >>>
> >>> Thanks
> >>>
> >>> Adam
> >>>
> >>>
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Jan
> >>>>
> >>>> With the Stores.windowStoreBuilder and Stores.persistentWindowStore,
> you
> >>>> actually only need to specify the amount of segments you want and how
> >> large
> >>>> they are. To the best of my understanding, what happens is that the
> >>>> segments are automatically rolled over as new data with new timestamps
> >> are
> >>>> created. We use this exact functionality in some of the work done
> >>>> internally at my company. For reference, this is the hopping windowed
> >> store.
> >>>>
> >>>>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>>>
> >>>> In the code that I have provided, there are going to be two 24h
> >> segments.
> >>>> When a record is put into the windowStore, it will be inserted at time
> >> T in
> >>>> both segments. The two segments will always overlap by 12h. As time
> >> goes on
> >>>> and new records are added (say at time T+12h+), the oldest segment
> will
> >> be
> >>>> automatically deleted and a new segment created. The records are by
> >> default
> >>>> inserted with the context.timestamp(), such that it is the record
> time,
> >> not
> >>>> the clock time, which is used.
> >>>>
> >>>> To the best of my understanding, the timestamps are retained when
> >>>> restoring from the changelog.
> >>>>
> >>>> Basically, this is heavy-handed way to deal with TTL at a
> segment-level,
> >>>> instead of at an individual record level.
> >>>>
> >>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <
> Jan.Filipiak@trivago.com>
> >>>> wrote:
> >>>>
> >>>>> Will that work? I expected it to blow up with ClassCastException or
> >>>>> similar.
> >>>>>
> >>>>> You either would have to specify the window you fetch/put or iterate
> >>>>> across all windows the key was found in right?
> >>>>>
> >>>>> I just hope the window-store doesn't check stream-time under the
> hoods
> >>>>> that would be a questionable interface.
> >>>>>
> >>>>> If it does: did you see my comment on checking all the windows
> earlier?
> >>>>> that would be needed to actually give reasonable time gurantees.
> >>>>>
> >>>>> Best
> >>>>>
> >>>>>
> >>>>>
> >>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>>>> Hi Jan
> >>>>>>
> >>>>>> Check for  " highwaterMat " in the PR. I only changed the state
> store,
> >>>>> not
> >>>>>> the ProcessorSupplier.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Adam
> >>>>>>
> >>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> >> Jan.Filipiak@trivago.com
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>>>
> >>>>>>>> @Guozhang
> >>>>>>>>
> >>>>>>>> Thanks for the information. This is indeed something that will be
> >>>>>>>> extremely
> >>>>>>>> useful for this KIP.
> >>>>>>>>
> >>>>>>>> @Jan
> >>>>>>>> Thanks for your explanations. That being said, I will not be
> moving
> >>>>> ahead
> >>>>>>>> with an implementation using reshuffle/groupBy solution as you
> >>>>> propose.
> >>>>>>>> That being said, if you wish to implement it yourself off of my
> >>>>> current PR
> >>>>>>>> and submit it as a competitive alternative, I would be more than
> >>>>> happy to
> >>>>>>>> help vet that as an alternate solution. As it stands right now, I
> do
> >>>>> not
> >>>>>>>> really have more time to invest into alternatives without there
> >> being
> >>>>> a
> >>>>>>>> strong indication from the binding voters which they would prefer.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> Hey, total no worries. I think I personally gave up on the streams
> >> DSL
> >>>>> for
> >>>>>>> some time already, otherwise I would have pulled this KIP through
> >>>>> already.
> >>>>>>> I am currently reimplementing my own DSL based on PAPI.
> >>>>>>>
> >>>>>>>
> >>>>>>>> I will look at finishing up my PR with the windowed state store in
> >> the
> >>>>>>>> next
> >>>>>>>> week or so, exercising it via tests, and then I will come back for
> >>>>> final
> >>>>>>>> discussions. In the meantime, I hope that any of the binding
> voters
> >>>>> could
> >>>>>>>> take a look at the KIP in the wiki. I have updated it according to
> >> the
> >>>>>>>> latest plan:
> >>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>>>> Support+non-key+joining+in+KTable
> >>>>>>>>
> >>>>>>>> I have also updated the KIP PR to use a windowed store. This could
> >> be
> >>>>>>>> replaced by the results of KIP-258 whenever they are completed.
> >>>>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>>
> >>>>>>>> Adam
> >>>>>>>>
> >>>>>>>
> >>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated in
> the
> >>>>> PR?
> >>>>>>> expected it to change to Windowed<K>,Long Missing something?
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <
> wangguoz@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as
> it
> >>>>> is
> >>>>>>>>> for
> >>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
> >>>>> want to
> >>>>>>>>> have "handling out-of-order data for source KTable" such that
> >>>>> instead of
> >>>>>>>>> blindly apply the updates to the materialized store, i.e.
> following
> >>>>>>>>> offset
> >>>>>>>>> ordering, we will reject updates that are older than the current
> >>>>> key's
> >>>>>>>>> timestamps, i.e. following timestamp ordering.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Guozhang
> >>>>>>>>>
> >>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Hello Adam,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
> >> high
> >>>>>>>>>> watermark store, now altered to be replaced with a window
> store),
> >> I
> >>>>>>>>>> think
> >>>>>>>>>> another current on-going KIP may actually help:
> >>>>>>>>>>
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> This is for adding the timestamp into a key-value store (i.e.
> only
> >>>>> for
> >>>>>>>>>> non-windowed KTable), and then one of its usage, as described in
> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we
> can
> >>>>> then
> >>>>>>>>>> "reject" updates from the source topics if its timestamp is
> >> smaller
> >>>>> than
> >>>>>>>>>> the current key's latest update timestamp. I think it is very
> >>>>> similar to
> >>>>>>>>>> what you have in mind for high watermark based filtering, while
> >> you
> >>>>> only
> >>>>>>>>>> need to make sure that the timestamps of the joining records are
> >>>>>>>>>>
> >>>>>>>>> correctly
> >>>>>>>>>
> >>>>>>>>>> inherited though the whole topology to the final stage.
> >>>>>>>>>>
> >>>>>>>>>> Note that this KIP is for key-value store and hence non-windowed
> >>>>> KTables
> >>>>>>>>>> only, but for windowed KTables we do not really have a good
> >> support
> >>>>> for
> >>>>>>>>>> their joins anyways (
> >>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>>>> I
> >>>>>>>>>> think we can just consider non-windowed KTable-KTable non-key
> >> joins
> >>>>> for
> >>>>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Guozhang
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >>>>> Jan.Filipiak@trivago.com
> >>>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hi Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>> Current highwater mark implementation would grow endlessly
> based
> >>>>> on
> >>>>>>>>>>>> primary key of original event. It is a pair of (<this table
> >>>>> primary
> >>>>>>>>>>>>
> >>>>>>>>>>> key>,
> >>>>>>>>>
> >>>>>>>>>> <highest offset seen for that key>). This is used to
> differentiate
> >>>>>>>>>>>>
> >>>>>>>>>>> between
> >>>>>>>>>
> >>>>>>>>>> late arrivals and new updates. My newest proposal would be to
> >>>>> replace
> >>>>>>>>>>>>
> >>>>>>>>>>> it
> >>>>>>>>>
> >>>>>>>>>> with a Windowed state store of Duration N. This would allow the
> >> same
> >>>>>>>>>>>> behaviour, but cap the size based on time. This should allow
> for
> >>>>> all
> >>>>>>>>>>>> late-arriving events to be processed, and should be
> customizable
> >>>>> by
> >>>>>>>>>>>> the
> >>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes
> >> of
> >>>>>>>>>>>>
> >>>>>>>>>>> window,
> >>>>>>>>>
> >>>>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi Adam, using time based retention can do the trick here.
> Even
> >>>>> if I
> >>>>>>>>>>> would still like to see the automatic repartitioning optional
> >>>>> since I
> >>>>>>>>>>>
> >>>>>>>>>> would
> >>>>>>>>>
> >>>>>>>>>> just reshuffle again. With windowed store I am a little bit
> >>>>> sceptical
> >>>>>>>>>>>
> >>>>>>>>>> about
> >>>>>>>>>
> >>>>>>>>>> how to determine the window. So esentially one could run into
> >>>>> problems
> >>>>>>>>>>>
> >>>>>>>>>> when
> >>>>>>>>>
> >>>>>>>>>> the rapid change happens near a window border. I will check you
> >>>>>>>>>>> implementation in detail, if its problematic, we could still
> >> check
> >>>>>>>>>>> _all_
> >>>>>>>>>>> windows on read with not to bad performance impact I guess.
> Will
> >>>>> let
> >>>>>>>>>>> you
> >>>>>>>>>>> know if the implementation would be correct as is. I wouldn't
> not
> >>>>> like
> >>>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>
> >>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> >> timestamp(B).
> >>>>> I
> >>>>>>>>>>>
> >>>>>>>>>> think
> >>>>>>>>>
> >>>>>>>>>> we can't expect that.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Jan
> >>>>>>>>>>>> I believe I understand what you mean now - thanks for the
> >>>>> diagram, it
> >>>>>>>>>>>> did really help. You are correct that I do not have the
> original
> >>>>>>>>>>>>
> >>>>>>>>>>> primary
> >>>>>>>>>
> >>>>>>>>>> key available, and I can see that if it was available then you
> >>>>> would be
> >>>>>>>>>>>> able to add and remove events from the Map. That being said, I
> >>>>>>>>>>>>
> >>>>>>>>>>> encourage
> >>>>>>>>>
> >>>>>>>>>> you to finish your diagrams / charts just for clarity for
> everyone
> >>>>>>>>>>>>
> >>>>>>>>>>> else.
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> >>>>> understand
> >>>>>>>>>>>>
> >>>>>>>>>>> the benefits for the rest. Sorry about the original primary
> key,
> >> We
> >>>>>>>>>>> have
> >>>>>>>>>>> join and Group by implemented our own in PAPI and basically not
> >>>>> using
> >>>>>>>>>>>
> >>>>>>>>>> any
> >>>>>>>>>
> >>>>>>>>>> DSL (Just the abstraction). Completely missed that in original
> DSL
> >>>>> its
> >>>>>>>>>>>
> >>>>>>>>>> not
> >>>>>>>>>
> >>>>>>>>>> there and just assumed it. total brain mess up on my end. Will
> >>>>> finish
> >>>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>
> >>>>>>>>>> chart as soon as i get a quite evening this week.
> >>>>>>>>>>>
> >>>>>>>>>>> My follow up question for you is, won't the Map stay inside the
> >>>>> State
> >>>>>>>>>>>
> >>>>>>>>>>>> Store indefinitely after all of the changes have propagated?
> >> Isn't
> >>>>>>>>>>>> this
> >>>>>>>>>>>> effectively the same as a highwater mark state store?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna return
> >>>>> `null`
> >>>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>
> >>>>>>>>>> the key is removed from the keyspace. But there is going to be a
> >>>>> store
> >>>>>>>>>>> 100%, the good thing is that I can use this store directly for
> >>>>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
> >>>>> satisfying
> >>>>>>>>>>> all gurantees needed for further groupby / join. The Windowed
> >>>>> store is
> >>>>>>>>>>>
> >>>>>>>>>> not
> >>>>>>>>>
> >>>>>>>>>> keeping the values, so for the next statefull operation we would
> >>>>>>>>>>> need to instantiate an extra store. or we have the window store
> >>>>> also
> >>>>>>>>>>>
> >>>>>>>>>> have
> >>>>>>>>>
> >>>>>>>>>> the values then.
> >>>>>>>>>>>
> >>>>>>>>>>> Long story short. if we can flip in a custom group by before
> >>>>>>>>>>> repartitioning to the original primary key i think it would
> help
> >>>>> the
> >>>>>>>>>>>
> >>>>>>>>>> users
> >>>>>>>>>
> >>>>>>>>>> big time in building efficient apps. Given the original primary
> >> key
> >>>>>>>>>>>
> >>>>>>>>>> issue I
> >>>>>>>>>
> >>>>>>>>>> understand that we do not have a solid foundation to build on.
> >>>>>>>>>>> Leaving primary key carry along to the user. very unfortunate.
> I
> >>>>> could
> >>>>>>>>>>> understand the decision goes like that. I do not think its a
> good
> >>>>>>>>>>>
> >>>>>>>>>> decision.
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks
> >>>>>>>>>>>> Adam
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:
> dumbreprajakta311@gmail.com
> >>>>
> >>>>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>>>         please remove me from this group
> >>>>>>>>>>>>
> >>>>>>>>>>>>         On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>         wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > Hi Adam,
> >>>>>>>>>>>>         >
> >>>>>>>>>>>>         > give me some time, will make such a chart. last
> time i
> >>>>> didn't
> >>>>>>>>>>>>         get along
> >>>>>>>>>>>>         > well with giphy and ruined all your charts.
> >>>>>>>>>>>>         > Hopefully i can get it done today
> >>>>>>>>>>>>         >
> >>>>>>>>>>>>         > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>>>>>>>>>>         > > Hi Jan
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > > I have included a diagram of what I attempted on
> the
> >>>>> KIP.
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         >
> >>>>>>>>>>>>
> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>>>         <
> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > > I attempted this back at the start of my own
> >>>>> implementation
> >>>>>>>>>>>> of
> >>>>>>>>>>>>         this
> >>>>>>>>>>>>         > > solution, and since I could not get it to work I
> have
> >>>>> since
> >>>>>>>>>>>>         discarded the
> >>>>>>>>>>>>         > > code. At this point in time, if you wish to
> continue
> >>>>> pursuing
> >>>>>>>>>>>>         for your
> >>>>>>>>>>>>         > > groupBy solution, I ask that you please create a
> >>>>> diagram on
> >>>>>>>>>>>>         the KIP
> >>>>>>>>>>>>         > > carefully explaining your solution. Please feel
> free
> >> to
> >>>>> use
> >>>>>>>>>>>>         the image I
> >>>>>>>>>>>>         > > just posted as a starting point. I am having
> trouble
> >>>>>>>>>>>>         understanding your
> >>>>>>>>>>>>         > > explanations but I think that a carefully
> constructed
> >>>>> diagram
> >>>>>>>>>>>>         will clear
> >>>>>>>>>>>>         > up
> >>>>>>>>>>>>         > > any misunderstandings. Alternately, please post a
> >>>>>>>>>>>>         comprehensive PR with
> >>>>>>>>>>>>         > > your solution. I can only guess at what you mean,
> and
> >>>>> since I
> >>>>>>>>>>>>         value my
> >>>>>>>>>>>>         > own
> >>>>>>>>>>>>         > > time as much as you value yours, I believe it is
> your
> >>>>>>>>>>>>         responsibility to
> >>>>>>>>>>>>         > > provide an implementation instead of me trying to
> >> guess.
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > > Adam
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > > wrote:
> >>>>>>>>>>>>         > >
> >>>>>>>>>>>>         > >> Hi James,
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >> nice to see you beeing interested. kafka streams
> at
> >>>>> this
> >>>>>>>>>>>>         point supports
> >>>>>>>>>>>>         > >> all sorts of joins as long as both streams have
> the
> >>>>> same
> >>>>>>>>>>>> key.
> >>>>>>>>>>>>         > >> Adam is currently implementing a join where a
> KTable
> >>>>> and a
> >>>>>>>>>>>>         KTable can
> >>>>>>>>>>>>         > have
> >>>>>>>>>>>>         > >> a one to many relation ship (1:n). We exploit
> that
> >>>>> rocksdb
> >>>>>>>>>>>> is
> >>>>>>>>>>>>
> >>>>>>>>>>> a
> >>>>>>>>>
> >>>>>>>>>>         > >> datastore that keeps data sorted (At least exposes
> an
> >>>>> API to
> >>>>>>>>>>>>         access the
> >>>>>>>>>>>>         > >> stored data in a sorted fashion).
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >> I think the technical caveats are well understood
> >> now
> >>>>> and we
> >>>>>>>>>>>>
> >>>>>>>>>>> are
> >>>>>>>>>
> >>>>>>>>>>         > basically
> >>>>>>>>>>>>         > >> down to philosophy and API Design ( when Adam
> sees
> >> my
> >>>>> newest
> >>>>>>>>>>>>         message).
> >>>>>>>>>>>>         > >> I have a lengthy track record of loosing those
> kinda
> >>>>>>>>>>>>         arguments within
> >>>>>>>>>>>>         > the
> >>>>>>>>>>>>         > >> streams community and I have no clue why. So I
> >>>>> literally
> >>>>>>>>>>>>         can't wait for
> >>>>>>>>>>>>         > you
> >>>>>>>>>>>>         > >> to churn through this thread and give you
> opinion on
> >>>>> how we
> >>>>>>>>>>>>         should
> >>>>>>>>>>>>         > design
> >>>>>>>>>>>>         > >> the return type of the oneToManyJoin and how many
> >>>>> power we
> >>>>>>>>>>>>         want to give
> >>>>>>>>>>>>         > to
> >>>>>>>>>>>>         > >> the user vs "simplicity" (where simplicity isn't
> >>>>> really that
> >>>>>>>>>>>>         as users
> >>>>>>>>>>>>         > still
> >>>>>>>>>>>>         > >> need to understand it I argue)
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >> waiting for you to join in on the discussion
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >> Best Jan
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>>>>>>>>>>         > >>
> >>>>>>>>>>>>         > >>> I am new to this group and I found this subject
> >>>>>>>>>>>>         interesting.  Sounds
> >>>>>>>>>>>>         > like
> >>>>>>>>>>>>         > >>> you guys want to implement a join table of two
> >>>>> streams? Is
> >>>>>>>>>>>> there
> >>>>>>>>>>>>         > somewhere
> >>>>>>>>>>>>         > >>> I can see the original requirement or proposal?
> >>>>>>>>>>>>         > >>>
> >>>>>>>>>>>>         > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > >>>> wrote:
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>> I'm currently testing using a Windowed Store
> to
> >>>>> store the
> >>>>>>>>>>>>         highwater
> >>>>>>>>>>>>         > >>>>> mark.
> >>>>>>>>>>>>         > >>>>> By all indications this should work fine, with
> >> the
> >>>>> caveat
> >>>>>>>>>>>>         being that
> >>>>>>>>>>>>         > it
> >>>>>>>>>>>>         > >>>>> can
> >>>>>>>>>>>>         > >>>>> only resolve out-of-order arrival for up to
> the
> >>>>> size of
> >>>>>>>>>>>>         the window
> >>>>>>>>>>>>         > (ie:
> >>>>>>>>>>>>         > >>>>> 24h, 72h, etc). This would remove the
> possibility
> >>>>> of it
> >>>>>>>>>>>>
> >>>>>>>>>>> being
> >>>>>>>>>
> >>>>>>>>>>         > unbounded
> >>>>>>>>>>>>         > >>>>> in
> >>>>>>>>>>>>         > >>>>> size.
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>> With regards to Jan's suggestion, I believe
> this
> >> is
> >>>>> where
> >>>>>>>>>>>>         we will
> >>>>>>>>>>>>         > have
> >>>>>>>>>>>>         > >>>>> to
> >>>>>>>>>>>>         > >>>>> remain in disagreement. While I do not
> disagree
> >>>>> with your
> >>>>>>>>>>>>         statement
> >>>>>>>>>>>>         > >>>>> about
> >>>>>>>>>>>>         > >>>>> there likely to be additional joins done in a
> >>>>> real-world
> >>>>>>>>>>>>         workflow, I
> >>>>>>>>>>>>         > do
> >>>>>>>>>>>>         > >>>>> not
> >>>>>>>>>>>>         > >>>>> see how you can conclusively deal with
> >> out-of-order
> >>>>>>>>>>>> arrival
> >>>>>>>>>>>> of
> >>>>>>>>>>>>         > >>>>> foreign-key
> >>>>>>>>>>>>         > >>>>> changes and subsequent joins. I have attempted
> >> what
> >>>>> I
> >>>>>>>>>>>>         think you have
> >>>>>>>>>>>>         > >>>>> proposed (without a high-water, using groupBy
> and
> >>>>> reduce)
> >>>>>>>>>>>>         and found
> >>>>>>>>>>>>         > >>>>> that if
> >>>>>>>>>>>>         > >>>>> the foreign key changes too quickly, or the
> load
> >> on
> >>>>> a
> >>>>>>>>>>>>         stream thread
> >>>>>>>>>>>>         > is
> >>>>>>>>>>>>         > >>>>> too
> >>>>>>>>>>>>         > >>>>> high, the joined messages will arrive
> >> out-of-order
> >>>>> and be
> >>>>>>>>>>>>         incorrectly
> >>>>>>>>>>>>         > >>>>> propagated, such that an intermediate event is
> >>>>>>>>>>>> represented
> >>>>>>>>>>>>         as the
> >>>>>>>>>>>>         > final
> >>>>>>>>>>>>         > >>>>> event.
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>> Can you shed some light on your groupBy
> >>>>> implementation.
> >>>>>>>>>>>>         There must be
> >>>>>>>>>>>>         > >>>> some sort of flaw in it.
> >>>>>>>>>>>>         > >>>> I have a suspicion where it is, I would just
> like
> >> to
> >>>>>>>>>>>>         confirm. The idea
> >>>>>>>>>>>>         > >>>> is bullet proof and it must be
> >>>>>>>>>>>>         > >>>> an implementation mess up. I would like to
> clarify
> >>>>> before
> >>>>>>>>>>>>         we draw a
> >>>>>>>>>>>>         > >>>> conclusion.
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>    Repartitioning the scattered events back to
> >> their
> >>>>>>>>>>>>
> >>>>>>>>>>> original
> >>>>>>>>>
> >>>>>>>>>>         > >>>>> partitions is the only way I know how to
> >> conclusively
> >>>>> deal
> >>>>>>>>>>>>         with
> >>>>>>>>>>>>         > >>>>> out-of-order events in a given time frame,
> and to
> >>>>> ensure
> >>>>>>>>>>>>         that the
> >>>>>>>>>>>>         > data
> >>>>>>>>>>>>         > >>>>> is
> >>>>>>>>>>>>         > >>>>> eventually consistent with the input events.
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>> If you have some code to share that
> illustrates
> >> your
> >>>>>>>>>>>>         approach, I
> >>>>>>>>>>>>         > would
> >>>>>>>>>>>>         > >>>>> be
> >>>>>>>>>>>>         > >>>>> very grateful as it would remove any
> >>>>> misunderstandings
> >>>>>>>>>>>>         that I may
> >>>>>>>>>>>>         > have.
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>> ah okay you were looking for my code. I don't
> have
> >>>>>>>>>>>>         something easily
> >>>>>>>>>>>>         > >>>> readable here as its bloated with OO-patterns.
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>> its anyhow trivial:
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>> @Override
> >>>>>>>>>>>>         > >>>>      public T apply(K aggKey, V value, T
> >> aggregate)
> >>>>>>>>>>>>         > >>>>      {
> >>>>>>>>>>>>         > >>>>          Map<U, V> currentStateAsMap =
> >>>>> asMap(aggregate);
> >>>>>>>>>>>> <<
> >>>>>>>>>>>>         imaginary
> >>>>>>>>>>>>         > >>>>          U toModifyKey = mapper.apply(value);
> >>>>>>>>>>>>         > >>>>              << this is the place where people
> >>>>> actually
> >>>>>>>>>>>>         gonna have
> >>>>>>>>>>>>         > issues
> >>>>>>>>>>>>         > >>>> and why you probably couldn't do it. we would
> need
> >>>>> to find
> >>>>>>>>>>>>         a solution
> >>>>>>>>>>>>         > here.
> >>>>>>>>>>>>         > >>>> I didn't realize that yet.
> >>>>>>>>>>>>         > >>>>              << we propagate the field in the
> >>>>> joiner, so
> >>>>>>>>>>>>         that we can
> >>>>>>>>>>>>         > pick
> >>>>>>>>>>>>         > >>>> it up in an aggregate. Probably you have not
> >> thought
> >>>>> of
> >>>>>>>>>>>>         this in your
> >>>>>>>>>>>>         > >>>> approach right?
> >>>>>>>>>>>>         > >>>>              << I am very open to find a
> generic
> >>>>> solution
> >>>>>>>>>>>>         here. In my
> >>>>>>>>>>>>         > >>>> honest opinion this is broken in
> >> KTableImpl.GroupBy
> >>>>> that
> >>>>>>>>>>>> it
> >>>>>>>>>>>>         looses
> >>>>>>>>>>>>         > the keys
> >>>>>>>>>>>>         > >>>> and only maintains the aggregate key.
> >>>>>>>>>>>>         > >>>>              << I abstracted it away back then
> way
> >>>>> before
> >>>>>>>>>>>> i
> >>>>>>>>>>>> was
> >>>>>>>>>>>>         > thinking
> >>>>>>>>>>>>         > >>>> of oneToMany join. That is why I didn't realize
> >> its
> >>>>>>>>>>>>         significance here.
> >>>>>>>>>>>>         > >>>>              << Opinions?
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>          for (V m : current)
> >>>>>>>>>>>>         > >>>>          {
> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >>>>>>>>>>>>         > >>>>          }
> >>>>>>>>>>>>         > >>>>          if (isAdder)
> >>>>>>>>>>>>         > >>>>          {
> >>>>>>>>>>>>         > >>>> currentStateAsMap.put(toModifyKey, value);
> >>>>>>>>>>>>         > >>>>          }
> >>>>>>>>>>>>         > >>>>          else
> >>>>>>>>>>>>         > >>>>          {
> >>>>>>>>>>>>         > >>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>>>         > >>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>>>         > >>>>                  return null;
> >>>>>>>>>>>>         > >>>>              }
> >>>>>>>>>>>>         > >>>>          }
> >>>>>>>>>>>>         > >>>>          retrun
> asAggregateType(currentStateAsMap)
> >>>>>>>>>>>>         > >>>>      }
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>>
> >>>>>>>>>>>>         > >>>> Thanks,
> >>>>>>>>>>>>         > >>>>> Adam
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> >>>>>>>>>>>>         > Jan.Filipiak@trivago.com <mailto:
> >> Jan.Filipiak@trivago.com
> >>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > >>>>> wrote:
> >>>>>>>>>>>>         > >>>>>
> >>>>>>>>>>>>         > >>>>> Thanks Adam for bringing Matthias to speed!
> >>>>>>>>>>>>         > >>>>>> about the differences. I think re-keying back
> >>>>> should be
> >>>>>>>>>>>>         optional at
> >>>>>>>>>>>>         > >>>>>> best.
> >>>>>>>>>>>>         > >>>>>> I would say we return a KScatteredTable with
> >>>>> reshuffle()
> >>>>>>>>>>>>         returning
> >>>>>>>>>>>>         > >>>>>> KTable<originalKey,Joined> to make the
> backwards
> >>>>>>>>>>>>         repartitioning
> >>>>>>>>>>>>         > >>>>>> optional.
> >>>>>>>>>>>>         > >>>>>> I am also in a big favour of doing the out of
> >> order
> >>>>>>>>>>>>         processing using
> >>>>>>>>>>>>         > >>>>>> group
> >>>>>>>>>>>>         > >>>>>> by instead high water mark tracking.
> >>>>>>>>>>>>         > >>>>>> Just because unbounded growth is just scary
> + It
> >>>>> saves
> >>>>>>>>>>>> us
> >>>>>>>>>>>>         the header
> >>>>>>>>>>>>         > >>>>>> stuff.
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>> I think the abstraction of always
> repartitioning
> >>>>> back is
> >>>>>>>>>>>>         just not so
> >>>>>>>>>>>>         > >>>>>> strong. Like the work has been done before we
> >>>>> partition
> >>>>>>>>>>>>         back and
> >>>>>>>>>>>>         > >>>>>> grouping
> >>>>>>>>>>>>         > >>>>>> by something else afterwards is really
> common.
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>>>>>>>>>>         > >>>>>>
> >>>>>>>>>>>>         > >>>>>> Hi Matthias
> >>>>>>>>>>>>         > >>>>>>> Thank you for your feedback, I do appreciate
> >> it!
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> While name spacing would be possible, it
> would
> >>>>> require
> >>>>>>>>>>>> to
> >>>>>>>>>>>>         > deserialize
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
> overhead.
> >> I
> >>>>> would
> >>>>>>>>>>>>         suggest to
> >>>>>>>>>>>>         > no
> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead. If
> >> this
> >>>>>>>>>>>>
> >>>>>>>>>>> becomes a
> >>>>>>>>>
> >>>>>>>>>>         > problem in
> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
> >> later
> >>>>> on.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Agreed. I will go with using a reserved
> string
> >>>>> and
> >>>>>>>>>>>>         document it.
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> My main concern about the design it the
> type of
> >>>>> the
> >>>>>>>>>>>>         result KTable:
> >>>>>>>>>>>>         > If
> >>>>>>>>>>>>         > >>>>>>> I
> >>>>>>>>>>>>         > >>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> In your example, you have table1 and table2
> >>>>> swapped.
> >>>>>>>>>>>>         Here is how it
> >>>>>>>>>>>>         > >>>>>>> works
> >>>>>>>>>>>>         > >>>>>>> currently:
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> 1) table1 has the records that contain the
> >>>>> foreign key
> >>>>>>>>>>>>         within their
> >>>>>>>>>>>>         > >>>>>>> value.
> >>>>>>>>>>>>         > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >>>>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>>>         > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>>>         > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> 2) A Value mapper is required to extract the
> >>>>> foreign
> >>>>>>>>>>>> key.
> >>>>>>>>>>>>         > >>>>>>> table1 foreign key mapper: ( value =>
> value.fk
> >>>>>>>>>>>>         <http://value.fk> )
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> The mapper is applied to each element in
> >> table1,
> >>>>> and a
> >>>>>>>>>>>>         new combined
> >>>>>>>>>>>>         > >>>>>>> key is
> >>>>>>>>>>>>         > >>>>>>> made:
> >>>>>>>>>>>>         > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> >>>>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>>>         <B-c,
> >>>>>>>>>>>>         > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> 3) The rekeyed events are copartitioned with
> >>>>> table2:
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>,
> <A-b,
> >>>>>>>>>>>>         (fk=A,bar=2)>
> >>>>>>>>>>>>         > >>>>>>> Table2: <A,X>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>>>>>>>>>>         > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> 4) From here, they can be joined together
> >> locally
> >>>>> by
> >>>>>>>>>>>>         applying the
> >>>>>>>>>>>>         > >>>>>>> joiner
> >>>>>>>>>>>>         > >>>>>>> function.
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> At this point, Jan's design and my design
> >>>>> deviate. My
> >>>>>>>>>>>>         design goes
> >>>>>>>>>>>>         > on
> >>>>>>>>>>>>         > >>>>>>> to
> >>>>>>>>>>>>         > >>>>>>> repartition the data post-join and resolve
> >>>>> out-of-order
> >>>>>>>>>>>>         arrival of
> >>>>>>>>>>>>         > >>>>>>> records,
> >>>>>>>>>>>>         > >>>>>>> finally returning the data keyed just the
> >>>>> original key.
> >>>>>>>>>>>>         I do not
> >>>>>>>>>>>>         > >>>>>>> expose
> >>>>>>>>>>>>         > >>>>>>> the
> >>>>>>>>>>>>         > >>>>>>> CombinedKey or any of the internals outside
> of
> >> the
> >>>>>>>>>>>>         joinOnForeignKey
> >>>>>>>>>>>>         > >>>>>>> function. This does make for larger
> footprint,
> >>>>> but it
> >>>>>>>>>>>>         removes all
> >>>>>>>>>>>>         > >>>>>>> agency
> >>>>>>>>>>>>         > >>>>>>> for resolving out-of-order arrivals and
> >> handling
> >>>>>>>>>>>>         CombinedKeys from
> >>>>>>>>>>>>         > the
> >>>>>>>>>>>>         > >>>>>>> user. I believe that this makes the function
> >> much
> >>>>>>>>>>>> easier
> >>>>>>>>>>>>         to use.
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> Let me know if this helps resolve your
> >> questions,
> >>>>> and
> >>>>>>>>>>>>         please feel
> >>>>>>>>>>>>         > >>>>>>> free to
> >>>>>>>>>>>>         > >>>>>>> add anything else on your mind.
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> Thanks again,
> >>>>>>>>>>>>         > >>>>>>> Adam
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
> >> Sax <
> >>>>>>>>>>>>         > >>>>>>> matthias@confluent.io <mailto:
> >>>>> matthias@confluent.io>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>> wrote:
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>> Hi,
> >>>>>>>>>>>>         > >>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> I am just catching up on this thread. I did
> >> not
> >>>>> read
> >>>>>>>>>>>>         everything so
> >>>>>>>>>>>>         > >>>>>>>> far,
> >>>>>>>>>>>>         > >>>>>>>> but want to share couple of initial
> thoughts:
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Headers: I think there is a fundamental
> >>>>> difference
> >>>>>>>>>>>>         between header
> >>>>>>>>>>>>         > >>>>>>>> usage
> >>>>>>>>>>>>         > >>>>>>>> in this KIP and KP-258. For 258, we add
> >> headers
> >>>>> to
> >>>>>>>>>>>>         changelog topic
> >>>>>>>>>>>>         > >>>>>>>> that
> >>>>>>>>>>>>         > >>>>>>>> are owned by Kafka Streams and nobody else
> is
> >>>>> supposed
> >>>>>>>>>>>>         to write
> >>>>>>>>>>>>         > into
> >>>>>>>>>>>>         > >>>>>>>> them. In fact, no user header are written
> into
> >>>>> the
> >>>>>>>>>>>>         changelog topic
> >>>>>>>>>>>>         > >>>>>>>> and
> >>>>>>>>>>>>         > >>>>>>>> thus, there are not conflicts.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Nevertheless, I don't see a big issue with
> >> using
> >>>>>>>>>>>>         headers within
> >>>>>>>>>>>>         > >>>>>>>> Streams.
> >>>>>>>>>>>>         > >>>>>>>> As long as we document it, we can have some
> >>>>> "reserved"
> >>>>>>>>>>>>         header keys
> >>>>>>>>>>>>         > >>>>>>>> and
> >>>>>>>>>>>>         > >>>>>>>> users are not allowed to use when
> processing
> >>>>> data with
> >>>>>>>>>>>>         Kafka
> >>>>>>>>>>>>         > Streams.
> >>>>>>>>>>>>         > >>>>>>>> IMHO, this should be ok.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> I think there is a safe way to avoid
> >> conflicts,
> >>>>> since
> >>>>>>>>>>>> these
> >>>>>>>>>>>>         > headers
> >>>>>>>>>>>>         > >>>>>>>> are
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>> only needed in internal topics (I think):
> >>>>>>>>>>>>         > >>>>>>>>> For internal and changelog topics, we can
> >>>>> namespace
> >>>>>>>>>>>>         all headers:
> >>>>>>>>>>>>         > >>>>>>>>> * user-defined headers are namespaced as
> >>>>> "external."
> >>>>>>>>>>>> +
> >>>>>>>>>>>>         headerKey
> >>>>>>>>>>>>         > >>>>>>>>> * internal headers are namespaced as
> >>>>> "internal." +
> >>>>>>>>>>>>         headerKey
> >>>>>>>>>>>>         > >>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>> While name spacing would be possible, it
> >> would
> >>>>>>>>>>>> require
> >>>>>>>>>>>>
> >>>>>>>>>>> to
> >>>>>>>>>
> >>>>>>>>>>         > >>>>>>>> deserialize
> >>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime
> overhead.
> >> I
> >>>>> would
> >>>>>>>>>>>>         suggest to
> >>>>>>>>>>>>         > no
> >>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead. If
> >> this
> >>>>>>>>>>>>
> >>>>>>>>>>> becomes a
> >>>>>>>>>
> >>>>>>>>>>         > problem in
> >>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
> >> later
> >>>>> on.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> My main concern about the design it the
> type
> >> of
> >>>>> the
> >>>>>>>>>>>>         result KTable:
> >>>>>>>>>>>>         > >>>>>>>> If I
> >>>>>>>>>>>>         > >>>>>>>> understood the proposal correctly,
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>>>>>>         > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> KTable<K1,V3> joinedTable =
> >>>>> table1.join(table2,...);
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> implies that the `joinedTable` has the same
> >> key
> >>>>> as the
> >>>>>>>>>>>>         left input
> >>>>>>>>>>>>         > >>>>>>>> table.
> >>>>>>>>>>>>         > >>>>>>>> IMHO, this does not work because if table2
> >>>>> contains
> >>>>>>>>>>>>         multiple rows
> >>>>>>>>>>>>         > >>>>>>>> that
> >>>>>>>>>>>>         > >>>>>>>> join with a record in table1 (what is the
> main
> >>>>> purpose
> >>>>>>>>>>>>
> >>>>>>>>>>> of
> >>>>>>>>>
> >>>>>>>>>> a
> >>>>>>>>>>>>         > foreign
> >>>>>>>>>>>>         > >>>>>>>> key
> >>>>>>>>>>>>         > >>>>>>>> join), the result table would only contain
> a
> >>>>> single
> >>>>>>>>>>>>         join result,
> >>>>>>>>>>>>         > but
> >>>>>>>>>>>>         > >>>>>>>> not
> >>>>>>>>>>>>         > >>>>>>>> multiple.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Example:
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> table1 input stream: <A,X>
> >>>>>>>>>>>>         > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> We use table2 value a foreign key to table1
> >> key
> >>>>> (ie,
> >>>>>>>>>>>>         "A" joins).
> >>>>>>>>>>>>         > If
> >>>>>>>>>>>>         > >>>>>>>> the
> >>>>>>>>>>>>         > >>>>>>>> result key is the same key as key of
> table1,
> >> this
> >>>>>>>>>>>>         implies that the
> >>>>>>>>>>>>         > >>>>>>>> result can either be <A, join(X,1)> or <A,
> >>>>> join(X,2)>
> >>>>>>>>>>>>         but not
> >>>>>>>>>>>>         > both.
> >>>>>>>>>>>>         > >>>>>>>> Because the share the same key, whatever
> >> result
> >>>>> record
> >>>>>>>>>>>>         we emit
> >>>>>>>>>>>>         > later,
> >>>>>>>>>>>>         > >>>>>>>> overwrite the previous result.
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> This is the reason why Jan originally
> proposed
> >>>>> to use
> >>>>>>>>>>>> a
> >>>>>>>>>>>>         > combination
> >>>>>>>>>>>>         > >>>>>>>> of
> >>>>>>>>>>>>         > >>>>>>>> both primary keys of the input tables as
> key
> >> of
> >>>>> the
> >>>>>>>>>>>>         output table.
> >>>>>>>>>>>>         > >>>>>>>> This
> >>>>>>>>>>>>         > >>>>>>>> makes the keys of the output table unique
> and
> >> we
> >>>>> can
> >>>>>>>>>>>>         store both in
> >>>>>>>>>>>>         > >>>>>>>> the
> >>>>>>>>>>>>         > >>>>>>>> output table:
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
> >>>>> join(X,2)>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Thoughts?
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> -Matthias
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>>>>>>>>         > >>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>> Just on remark here.
> >>>>>>>>>>>>         > >>>>>>>>> The high-watermark could be disregarded.
> The
> >>>>> decision
> >>>>>>>>>>>>         about the
> >>>>>>>>>>>>         > >>>>>>>>> forward
> >>>>>>>>>>>>         > >>>>>>>>> depends on the size of the aggregated map.
> >>>>>>>>>>>>         > >>>>>>>>> Only 1 element long maps would be unpacked
> >> and
> >>>>>>>>>>>>         forwarded. 0
> >>>>>>>>>>>>         > element
> >>>>>>>>>>>>         > >>>>>>>>> maps
> >>>>>>>>>>>>         > >>>>>>>>> would be published as delete. Any other
> count
> >>>>>>>>>>>>         > >>>>>>>>> of map entries is in "waiting for correct
> >>>>> deletes to
> >>>>>>>>>>>>         > arrive"-state.
> >>>>>>>>>>>>         > >>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>>>>>>>         > >>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>> It does look like I could replace the
> second
> >>>>>>>>>>>>         repartition store
> >>>>>>>>>>>>         > and
> >>>>>>>>>>>>         > >>>>>>>>>> highwater store with a groupBy and
> reduce.
> >>>>> However,
> >>>>>>>>>>>>         it looks
> >>>>>>>>>>>>         > like
> >>>>>>>>>>>>         > >>>>>>>>>> I
> >>>>>>>>>>>>         > >>>>>>>>>> would
> >>>>>>>>>>>>         > >>>>>>>>>> still need to store the highwater value
> >> within
> >>>>> the
> >>>>>>>>>>>>         materialized
> >>>>>>>>>>>>         > >>>>>>>>>> store,
> >>>>>>>>>>>>         > >>>>>>>>>>
> >>>>>>>>>>>>         > >>>>>>>>>> to
> >>>>>>>>>>>>         > >>>>>>>>> compare the arrival of out-of-order
> records
> >>>>> (assuming
> >>>>>>>>>>>>
> >>>>>>>>>>> my
> >>>>>>>>>
> >>>>>>>>>>         > >>>>>>>>> understanding
> >>>>>>>>>>>>         > >>>>>>>>> of
> >>>>>>>>>>>>         > >>>>>>>>> THIS is correct...). This in effect is the
> >> same
> >>>>> as
> >>>>>>>>>>>> the
> >>>>>>>>>>>>         design I
> >>>>>>>>>>>>         > have
> >>>>>>>>>>>>         > >>>>>>>>> now,
> >>>>>>>>>>>>         > >>>>>>>>> just with the two tables merged together.
> >>>>>>>>>>>>         > >>>>>>>>>
> >>>>>>>>>>>>         >
> >>>>>>>>>>>>         >
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> --
> >>>>>>>>>> -- Guozhang
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> --
> >>>>>>>>> -- Guozhang
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

On 29.11.2018 15:14, John Roesler wrote:
> Hi all,
>
> Sorry that this discussion petered out... I think the 2.1 release caused an
> extended distraction that pushed it off everyone's radar (which was
> precisely Adam's concern). Personally, I've also had some extend
> distractions of my own that kept (and continue to keep) me preoccupied.
>
> However, calling for a vote did wake me up, so I guess Jan was on the right
> track!
>
> I've gone back and reviewed the whole KIP document and the prior
> discussion, and I'd like to offer a few thoughts:
>
> API Thoughts:
>
> 1. If I read the KIP right, you are proposing a many-to-one join. Could we
> consider naming it manyToOneJoin? Or, if you prefer, flip the design around
> and make it a oneToManyJoin?
>
> The proposed name "joinOnForeignKey" disguises the join type, and it seems
> like it might trick some people into using it for a one-to-one join. This
> would work, of course, but it would be super inefficient compared to a
> simple rekey-and-join.
>
> 2. I might have missed it, but I don't think it's specified whether it's an
> inner, outer, or left join. I'm guessing an outer join, as (neglecting IQ),
> the rest can be achieved by filtering or by handling it in the ValueJoiner.
>
> 3. The arg list to joinOnForeignKey doesn't look quite right.
> 3a. Regarding Serialized: There are a few different paradigms in play in
> the Streams API, so it's confusing, but instead of three Serialized args, I
> think it would be better to have one that allows (optionally) setting the 4
> incoming serdes. The result serde is defined by the Materialized. The
> incoming serdes can be optional because they might already be available on
> the source KTables, or the default serdes from the config might be
> applicable.
>
> 3b. Is the StreamPartitioner necessary? The other joins don't allow setting
> one, and it seems like it might actually be harmful, since the rekey
> operation needs to produce results that are co-partitioned with the "other"
> KTable.
>
> 4. I'm fine with the "reserved word" header, but I didn't actually follow
> what Matthias meant about namespacing requiring "deserializing" the record
> header. The headers are already Strings, so I don't think that
> deserialization is required. If we applied the namespace at source nodes
> and stripped it at sink nodes, this would be practically no overhead. The
> advantage of the namespace idea is that no public API change wrt headers
> needs to happen, and no restrictions need to be placed on users' headers.
>
> (Although I'm wondering if we can get away without the header at all...
> stay tuned)
>
> 5. I also didn't follow the discussion about the HWM table growing without
> bound. As I read it, the HWM table is effectively implementing OCC to
> resolve the problem you noted with disordering when the rekey is
> reversed... particularly notable when the FK changes. As such, it only
> needs to track the most recent "version" (the offset in the source
> partition) of each key. Therefore, it should have the same number of keys
> as the source table at all times.
>
> I see that you are aware of KIP-258, which I think might be relevant in a
> couple of ways. One: it's just about storing the timestamp in the state
> store, but the ultimate idea is to effectively use the timestamp as an OCC
> "version" to drop disordered updates. You wouldn't want to use the
> timestamp for this operation, but if you were to use a similar mechanism to
> store the source offset in the store alongside the re-keyed values, then
> you could avoid a separate table.
>
> 6. You and Jan have been thinking about this for a long time, so I've
> probably missed something here, but I'm wondering if we can avoid the HWM
> tracking at all and resolve out-of-order during a final join instead...
>
> Let's say we're joining a left table (Integer K: Letter FK, (other data))
> to a right table (Letter K: (some data)).
>
> Left table:
> 1: (A, xyz)
> 2: (B, asd)
>
> Right table:
> A: EntityA
> B: EntityB
>
> We could do a rekey as you proposed with a combined key, but not
> propagating the value at all..
> Rekey table:
> A-1: (dummy value)
> B-2: (dummy value)
>
> Which we then join with the right table to produce:
> A-1: EntityA
> B-2: EntityB
>
> Which gets rekeyed back:
> 1: A, EntityA
> 2: B, EntityB
>
> And finally we do the actual join:
> Result table:
> 1: ((A, xyz), EntityA)
> 2: ((B, asd), EntityB)
>
> The thing is that in that last join, we have the opportunity to compare the
> current FK in the left table with the incoming PK of the right table. If
> they don't match, we just drop the event, since it must be outdated.
>

> In your KIP, you gave an example in which (1: A, xyz) gets updated to (1:
> B, xyz), ultimately yielding a conundrum about whether the final state
> should be (1: null) or (1: joined-on-B). With the algorithm above, you
> would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
> EntityB)). It seems like this does give you enough information to make the
> right choice, regardless of disordering.

Will check Adams patch, but this should work. As mentioned often I am 
not convinced on partitioning back for the user automatically. I think 
this is the real performance eater ;)

>
>
> 7. Last thought... I'm a little concerned about the performance of the
> range scans when records change in the right table. You've said that you've
> been using the algorithm you presented in production for a while. Can you
> give us a sense of the performance characteristics you've observed?
>

Make it work, make it fast, make it beautiful. The topmost thing here is 
/ was correctness. In practice I do not measure the performance of the 
range scan. Usual cases I run this with is emitting 500k - 1kk rows
on a left hand side change. The range scan is just the work you gotta 
do, also when you pack your data into different formats, usually the 
rocks performance is very tight to the size of the data and we can't 
really change that. It is more important for users to prevent useless 
updates to begin with. My left hand side is guarded to drop changes that 
are not going to change my join output.

usually it's:

drop unused fields and then don't forward if old.equals(new)

regarding to the performance of creating an iterator for smaller 
fanouts, users can still just do a group by first then anyways.



> I could only think of one alternative, but I'm not sure if it's better or
> worse... If the first re-key only needs to preserve the original key, as I
> proposed in #6, then we could store a vector of keys in the value:
>
> Left table:
> 1: A,...
> 2: B,...
> 3: A,...
>
> Gets re-keyed:
> A: [1, 3]
> B: [2]
>
> Then, the rhs part of the join would only need a regular single-key lookup.
> Of course we have to deal with the problem of large values, as there's no
> bound on the number of lhs records that can reference rhs records. Offhand,
> I'd say we could page the values, so when one row is past the threshold, we
> append the key for the next page. Then in most cases, it would be a single
> key lookup, but for large fan-out updates, it would be one per (max value
> size)/(avg lhs key size).
>
> This seems more complex, though... Plus, I think there's some extra
> tracking we'd need to do to know when to emit a retraction. For example,
> when record 1 is deleted, the re-key table would just have (A: [3]). Some
> kind of tombstone is needed so that the join result for 1 can also be
> retracted.
>
> That's all!
>
> Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
> discussion has been slow.
> -John
>
>
> On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Id say you can just call the vote.
>>
>> that happens all the time, and if something comes up, it just goes back
>> to discuss.
>>
>> would not expect to much attention with another another email in this
>> thread.
>>
>> best Jan
>>
>> On 09.10.2018 13:56, Adam Bellemare wrote:
>>> Hello Contributors
>>>
>>> I know that 2.1 is about to be released, but I do need to bump this to
>> keep
>>> visibility up. I am still intending to push this through once contributor
>>> feedback is given.
>>>
>>> Main points that need addressing:
>>> 1) Any way (or benefit) in structuring the current singular graph node
>> into
>>> multiple nodes? It has a whopping 25 parameters right now. I am a bit
>> fuzzy
>>> on how the optimizations are supposed to work, so I would appreciate any
>>> help on this aspect.
>>>
>>> 2) Overall strategy for joining + resolving. This thread has much
>> discourse
>>> between Jan and I between the current highwater mark proposal and a
>> groupBy
>>> + reduce proposal. I am of the opinion that we need to strictly handle
>> any
>>> chance of out-of-order data and leave none of it up to the consumer. Any
>>> comments or suggestions here would also help.
>>>
>>> 3) Anything else that you see that would prevent this from moving to a
>> vote?
>>>
>>> Thanks
>>>
>>> Adam
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>> wrote:
>>>
>>>> Hi Jan
>>>>
>>>> With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you
>>>> actually only need to specify the amount of segments you want and how
>> large
>>>> they are. To the best of my understanding, what happens is that the
>>>> segments are automatically rolled over as new data with new timestamps
>> are
>>>> created. We use this exact functionality in some of the work done
>>>> internally at my company. For reference, this is the hopping windowed
>> store.
>>>>
>>>>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>>>
>>>> In the code that I have provided, there are going to be two 24h
>> segments.
>>>> When a record is put into the windowStore, it will be inserted at time
>> T in
>>>> both segments. The two segments will always overlap by 12h. As time
>> goes on
>>>> and new records are added (say at time T+12h+), the oldest segment will
>> be
>>>> automatically deleted and a new segment created. The records are by
>> default
>>>> inserted with the context.timestamp(), such that it is the record time,
>> not
>>>> the clock time, which is used.
>>>>
>>>> To the best of my understanding, the timestamps are retained when
>>>> restoring from the changelog.
>>>>
>>>> Basically, this is heavy-handed way to deal with TTL at a segment-level,
>>>> instead of at an individual record level.
>>>>
>>>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>> Will that work? I expected it to blow up with ClassCastException or
>>>>> similar.
>>>>>
>>>>> You either would have to specify the window you fetch/put or iterate
>>>>> across all windows the key was found in right?
>>>>>
>>>>> I just hope the window-store doesn't check stream-time under the hoods
>>>>> that would be a questionable interface.
>>>>>
>>>>> If it does: did you see my comment on checking all the windows earlier?
>>>>> that would be needed to actually give reasonable time gurantees.
>>>>>
>>>>> Best
>>>>>
>>>>>
>>>>>
>>>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>>>> Hi Jan
>>>>>>
>>>>>> Check for  " highwaterMat " in the PR. I only changed the state store,
>>>>> not
>>>>>> the ProcessorSupplier.
>>>>>>
>>>>>> Thanks,
>>>>>> Adam
>>>>>>
>>>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
>> Jan.Filipiak@trivago.com
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>>>
>>>>>>>> @Guozhang
>>>>>>>>
>>>>>>>> Thanks for the information. This is indeed something that will be
>>>>>>>> extremely
>>>>>>>> useful for this KIP.
>>>>>>>>
>>>>>>>> @Jan
>>>>>>>> Thanks for your explanations. That being said, I will not be moving
>>>>> ahead
>>>>>>>> with an implementation using reshuffle/groupBy solution as you
>>>>> propose.
>>>>>>>> That being said, if you wish to implement it yourself off of my
>>>>> current PR
>>>>>>>> and submit it as a competitive alternative, I would be more than
>>>>> happy to
>>>>>>>> help vet that as an alternate solution. As it stands right now, I do
>>>>> not
>>>>>>>> really have more time to invest into alternatives without there
>> being
>>>>> a
>>>>>>>> strong indication from the binding voters which they would prefer.
>>>>>>>>
>>>>>>>>
>>>>>>> Hey, total no worries. I think I personally gave up on the streams
>> DSL
>>>>> for
>>>>>>> some time already, otherwise I would have pulled this KIP through
>>>>> already.
>>>>>>> I am currently reimplementing my own DSL based on PAPI.
>>>>>>>
>>>>>>>
>>>>>>>> I will look at finishing up my PR with the windowed state store in
>> the
>>>>>>>> next
>>>>>>>> week or so, exercising it via tests, and then I will come back for
>>>>> final
>>>>>>>> discussions. In the meantime, I hope that any of the binding voters
>>>>> could
>>>>>>>> take a look at the KIP in the wiki. I have updated it according to
>> the
>>>>>>>> latest plan:
>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>>>> Support+non-key+joining+in+KTable
>>>>>>>>
>>>>>>>> I have also updated the KIP PR to use a windowed store. This could
>> be
>>>>>>>> replaced by the results of KIP-258 whenever they are completed.
>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>>
>>>>>>>> Adam
>>>>>>>>
>>>>>>>
>>>>>>> Is the HighWatermarkResolverProccessorsupplier already updated in the
>>>>> PR?
>>>>>>> expected it to change to Windowed<K>,Long Missing something?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as it
>>>>> is
>>>>>>>>> for
>>>>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
>>>>> want to
>>>>>>>>> have "handling out-of-order data for source KTable" such that
>>>>> instead of
>>>>>>>>> blindly apply the updates to the materialized store, i.e. following
>>>>>>>>> offset
>>>>>>>>> ordering, we will reject updates that are older than the current
>>>>> key's
>>>>>>>>> timestamps, i.e. following timestamp ordering.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
>> wangguoz@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Hello Adam,
>>>>>>>>>>
>>>>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
>> high
>>>>>>>>>> watermark store, now altered to be replaced with a window store),
>> I
>>>>>>>>>> think
>>>>>>>>>> another current on-going KIP may actually help:
>>>>>>>>>>
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> This is for adding the timestamp into a key-value store (i.e. only
>>>>> for
>>>>>>>>>> non-windowed KTable), and then one of its usage, as described in
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
>>>>> then
>>>>>>>>>> "reject" updates from the source topics if its timestamp is
>> smaller
>>>>> than
>>>>>>>>>> the current key's latest update timestamp. I think it is very
>>>>> similar to
>>>>>>>>>> what you have in mind for high watermark based filtering, while
>> you
>>>>> only
>>>>>>>>>> need to make sure that the timestamps of the joining records are
>>>>>>>>>>
>>>>>>>>> correctly
>>>>>>>>>
>>>>>>>>>> inherited though the whole topology to the final stage.
>>>>>>>>>>
>>>>>>>>>> Note that this KIP is for key-value store and hence non-windowed
>>>>> KTables
>>>>>>>>>> only, but for windowed KTables we do not really have a good
>> support
>>>>> for
>>>>>>>>>> their joins anyways (
>>>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>>>> I
>>>>>>>>>> think we can just consider non-windowed KTable-KTable non-key
>> joins
>>>>> for
>>>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>>>
>>>>>>>>>>> Hi Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>> Current highwater mark implementation would grow endlessly based
>>>>> on
>>>>>>>>>>>> primary key of original event. It is a pair of (<this table
>>>>> primary
>>>>>>>>>>>>
>>>>>>>>>>> key>,
>>>>>>>>>
>>>>>>>>>> <highest offset seen for that key>). This is used to differentiate
>>>>>>>>>>>>
>>>>>>>>>>> between
>>>>>>>>>
>>>>>>>>>> late arrivals and new updates. My newest proposal would be to
>>>>> replace
>>>>>>>>>>>>
>>>>>>>>>>> it
>>>>>>>>>
>>>>>>>>>> with a Windowed state store of Duration N. This would allow the
>> same
>>>>>>>>>>>> behaviour, but cap the size based on time. This should allow for
>>>>> all
>>>>>>>>>>>> late-arriving events to be processed, and should be customizable
>>>>> by
>>>>>>>>>>>> the
>>>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes
>> of
>>>>>>>>>>>>
>>>>>>>>>>> window,
>>>>>>>>>
>>>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>>>
>>>>>>>>>>>> Hi Adam, using time based retention can do the trick here. Even
>>>>> if I
>>>>>>>>>>> would still like to see the automatic repartitioning optional
>>>>> since I
>>>>>>>>>>>
>>>>>>>>>> would
>>>>>>>>>
>>>>>>>>>> just reshuffle again. With windowed store I am a little bit
>>>>> sceptical
>>>>>>>>>>>
>>>>>>>>>> about
>>>>>>>>>
>>>>>>>>>> how to determine the window. So esentially one could run into
>>>>> problems
>>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>
>>>>>>>>>> the rapid change happens near a window border. I will check you
>>>>>>>>>>> implementation in detail, if its problematic, we could still
>> check
>>>>>>>>>>> _all_
>>>>>>>>>>> windows on read with not to bad performance impact I guess. Will
>>>>> let
>>>>>>>>>>> you
>>>>>>>>>>> know if the implementation would be correct as is. I wouldn't not
>>>>> like
>>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>
>>>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
>> timestamp(B).
>>>>> I
>>>>>>>>>>>
>>>>>>>>>> think
>>>>>>>>>
>>>>>>>>>> we can't expect that.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> @Jan
>>>>>>>>>>>> I believe I understand what you mean now - thanks for the
>>>>> diagram, it
>>>>>>>>>>>> did really help. You are correct that I do not have the original
>>>>>>>>>>>>
>>>>>>>>>>> primary
>>>>>>>>>
>>>>>>>>>> key available, and I can see that if it was available then you
>>>>> would be
>>>>>>>>>>>> able to add and remove events from the Map. That being said, I
>>>>>>>>>>>>
>>>>>>>>>>> encourage
>>>>>>>>>
>>>>>>>>>> you to finish your diagrams / charts just for clarity for everyone
>>>>>>>>>>>>
>>>>>>>>>>> else.
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
>>>>> understand
>>>>>>>>>>>>
>>>>>>>>>>> the benefits for the rest. Sorry about the original primary key,
>> We
>>>>>>>>>>> have
>>>>>>>>>>> join and Group by implemented our own in PAPI and basically not
>>>>> using
>>>>>>>>>>>
>>>>>>>>>> any
>>>>>>>>>
>>>>>>>>>> DSL (Just the abstraction). Completely missed that in original DSL
>>>>> its
>>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>
>>>>>>>>>> there and just assumed it. total brain mess up on my end. Will
>>>>> finish
>>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> chart as soon as i get a quite evening this week.
>>>>>>>>>>>
>>>>>>>>>>> My follow up question for you is, won't the Map stay inside the
>>>>> State
>>>>>>>>>>>
>>>>>>>>>>>> Store indefinitely after all of the changes have propagated?
>> Isn't
>>>>>>>>>>>> this
>>>>>>>>>>>> effectively the same as a highwater mark state store?
>>>>>>>>>>>>
>>>>>>>>>>>> Thing is that if the map is empty, substractor is gonna return
>>>>> `null`
>>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>
>>>>>>>>>> the key is removed from the keyspace. But there is going to be a
>>>>> store
>>>>>>>>>>> 100%, the good thing is that I can use this store directly for
>>>>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
>>>>> satisfying
>>>>>>>>>>> all gurantees needed for further groupby / join. The Windowed
>>>>> store is
>>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>
>>>>>>>>>> keeping the values, so for the next statefull operation we would
>>>>>>>>>>> need to instantiate an extra store. or we have the window store
>>>>> also
>>>>>>>>>>>
>>>>>>>>>> have
>>>>>>>>>
>>>>>>>>>> the values then.
>>>>>>>>>>>
>>>>>>>>>>> Long story short. if we can flip in a custom group by before
>>>>>>>>>>> repartitioning to the original primary key i think it would help
>>>>> the
>>>>>>>>>>>
>>>>>>>>>> users
>>>>>>>>>
>>>>>>>>>> big time in building efficient apps. Given the original primary
>> key
>>>>>>>>>>>
>>>>>>>>>> issue I
>>>>>>>>>
>>>>>>>>>> understand that we do not have a solid foundation to build on.
>>>>>>>>>>> Leaving primary key carry along to the user. very unfortunate. I
>>>>> could
>>>>>>>>>>> understand the decision goes like that. I do not think its a good
>>>>>>>>>>>
>>>>>>>>>> decision.
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks
>>>>>>>>>>>> Adam
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>>>>>>>> dumbreprajakta311@gmail.com <mailto:dumbreprajakta311@gmail.com
>>>>
>>>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>>         please remove me from this group
>>>>>>>>>>>>
>>>>>>>>>>>>         On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>         > Hi Adam,
>>>>>>>>>>>>         >
>>>>>>>>>>>>         > give me some time, will make such a chart. last time i
>>>>> didn't
>>>>>>>>>>>>         get along
>>>>>>>>>>>>         > well with giphy and ruined all your charts.
>>>>>>>>>>>>         > Hopefully i can get it done today
>>>>>>>>>>>>         >
>>>>>>>>>>>>         > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>>>>>>>         > > Hi Jan
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > > I have included a diagram of what I attempted on the
>>>>> KIP.
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         >
>>>>>>>>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>>>         <
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > > I attempted this back at the start of my own
>>>>> implementation
>>>>>>>>>>>> of
>>>>>>>>>>>>         this
>>>>>>>>>>>>         > > solution, and since I could not get it to work I have
>>>>> since
>>>>>>>>>>>>         discarded the
>>>>>>>>>>>>         > > code. At this point in time, if you wish to continue
>>>>> pursuing
>>>>>>>>>>>>         for your
>>>>>>>>>>>>         > > groupBy solution, I ask that you please create a
>>>>> diagram on
>>>>>>>>>>>>         the KIP
>>>>>>>>>>>>         > > carefully explaining your solution. Please feel free
>> to
>>>>> use
>>>>>>>>>>>>         the image I
>>>>>>>>>>>>         > > just posted as a starting point. I am having trouble
>>>>>>>>>>>>         understanding your
>>>>>>>>>>>>         > > explanations but I think that a carefully constructed
>>>>> diagram
>>>>>>>>>>>>         will clear
>>>>>>>>>>>>         > up
>>>>>>>>>>>>         > > any misunderstandings. Alternately, please post a
>>>>>>>>>>>>         comprehensive PR with
>>>>>>>>>>>>         > > your solution. I can only guess at what you mean, and
>>>>> since I
>>>>>>>>>>>>         value my
>>>>>>>>>>>>         > own
>>>>>>>>>>>>         > > time as much as you value yours, I believe it is your
>>>>>>>>>>>>         responsibility to
>>>>>>>>>>>>         > > provide an implementation instead of me trying to
>> guess.
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > > Adam
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>         > > wrote:
>>>>>>>>>>>>         > >
>>>>>>>>>>>>         > >> Hi James,
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >> nice to see you beeing interested. kafka streams at
>>>>> this
>>>>>>>>>>>>         point supports
>>>>>>>>>>>>         > >> all sorts of joins as long as both streams have the
>>>>> same
>>>>>>>>>>>> key.
>>>>>>>>>>>>         > >> Adam is currently implementing a join where a KTable
>>>>> and a
>>>>>>>>>>>>         KTable can
>>>>>>>>>>>>         > have
>>>>>>>>>>>>         > >> a one to many relation ship (1:n). We exploit that
>>>>> rocksdb
>>>>>>>>>>>> is
>>>>>>>>>>>>
>>>>>>>>>>> a
>>>>>>>>>
>>>>>>>>>>         > >> datastore that keeps data sorted (At least exposes an
>>>>> API to
>>>>>>>>>>>>         access the
>>>>>>>>>>>>         > >> stored data in a sorted fashion).
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >> I think the technical caveats are well understood
>> now
>>>>> and we
>>>>>>>>>>>>
>>>>>>>>>>> are
>>>>>>>>>
>>>>>>>>>>         > basically
>>>>>>>>>>>>         > >> down to philosophy and API Design ( when Adam sees
>> my
>>>>> newest
>>>>>>>>>>>>         message).
>>>>>>>>>>>>         > >> I have a lengthy track record of loosing those kinda
>>>>>>>>>>>>         arguments within
>>>>>>>>>>>>         > the
>>>>>>>>>>>>         > >> streams community and I have no clue why. So I
>>>>> literally
>>>>>>>>>>>>         can't wait for
>>>>>>>>>>>>         > you
>>>>>>>>>>>>         > >> to churn through this thread and give you opinion on
>>>>> how we
>>>>>>>>>>>>         should
>>>>>>>>>>>>         > design
>>>>>>>>>>>>         > >> the return type of the oneToManyJoin and how many
>>>>> power we
>>>>>>>>>>>>         want to give
>>>>>>>>>>>>         > to
>>>>>>>>>>>>         > >> the user vs "simplicity" (where simplicity isn't
>>>>> really that
>>>>>>>>>>>>         as users
>>>>>>>>>>>>         > still
>>>>>>>>>>>>         > >> need to understand it I argue)
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >> waiting for you to join in on the discussion
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >> Best Jan
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>>>>>>>         > >>
>>>>>>>>>>>>         > >>> I am new to this group and I found this subject
>>>>>>>>>>>>         interesting.  Sounds
>>>>>>>>>>>>         > like
>>>>>>>>>>>>         > >>> you guys want to implement a join table of two
>>>>> streams? Is
>>>>>>>>>>>> there
>>>>>>>>>>>>         > somewhere
>>>>>>>>>>>>         > >>> I can see the original requirement or proposal?
>>>>>>>>>>>>         > >>>
>>>>>>>>>>>>         > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>>>>>>>         <Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>         > >>>> wrote:
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>> I'm currently testing using a Windowed Store to
>>>>> store the
>>>>>>>>>>>>         highwater
>>>>>>>>>>>>         > >>>>> mark.
>>>>>>>>>>>>         > >>>>> By all indications this should work fine, with
>> the
>>>>> caveat
>>>>>>>>>>>>         being that
>>>>>>>>>>>>         > it
>>>>>>>>>>>>         > >>>>> can
>>>>>>>>>>>>         > >>>>> only resolve out-of-order arrival for up to the
>>>>> size of
>>>>>>>>>>>>         the window
>>>>>>>>>>>>         > (ie:
>>>>>>>>>>>>         > >>>>> 24h, 72h, etc). This would remove the possibility
>>>>> of it
>>>>>>>>>>>>
>>>>>>>>>>> being
>>>>>>>>>
>>>>>>>>>>         > unbounded
>>>>>>>>>>>>         > >>>>> in
>>>>>>>>>>>>         > >>>>> size.
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>> With regards to Jan's suggestion, I believe this
>> is
>>>>> where
>>>>>>>>>>>>         we will
>>>>>>>>>>>>         > have
>>>>>>>>>>>>         > >>>>> to
>>>>>>>>>>>>         > >>>>> remain in disagreement. While I do not disagree
>>>>> with your
>>>>>>>>>>>>         statement
>>>>>>>>>>>>         > >>>>> about
>>>>>>>>>>>>         > >>>>> there likely to be additional joins done in a
>>>>> real-world
>>>>>>>>>>>>         workflow, I
>>>>>>>>>>>>         > do
>>>>>>>>>>>>         > >>>>> not
>>>>>>>>>>>>         > >>>>> see how you can conclusively deal with
>> out-of-order
>>>>>>>>>>>> arrival
>>>>>>>>>>>> of
>>>>>>>>>>>>         > >>>>> foreign-key
>>>>>>>>>>>>         > >>>>> changes and subsequent joins. I have attempted
>> what
>>>>> I
>>>>>>>>>>>>         think you have
>>>>>>>>>>>>         > >>>>> proposed (without a high-water, using groupBy and
>>>>> reduce)
>>>>>>>>>>>>         and found
>>>>>>>>>>>>         > >>>>> that if
>>>>>>>>>>>>         > >>>>> the foreign key changes too quickly, or the load
>> on
>>>>> a
>>>>>>>>>>>>         stream thread
>>>>>>>>>>>>         > is
>>>>>>>>>>>>         > >>>>> too
>>>>>>>>>>>>         > >>>>> high, the joined messages will arrive
>> out-of-order
>>>>> and be
>>>>>>>>>>>>         incorrectly
>>>>>>>>>>>>         > >>>>> propagated, such that an intermediate event is
>>>>>>>>>>>> represented
>>>>>>>>>>>>         as the
>>>>>>>>>>>>         > final
>>>>>>>>>>>>         > >>>>> event.
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>> Can you shed some light on your groupBy
>>>>> implementation.
>>>>>>>>>>>>         There must be
>>>>>>>>>>>>         > >>>> some sort of flaw in it.
>>>>>>>>>>>>         > >>>> I have a suspicion where it is, I would just like
>> to
>>>>>>>>>>>>         confirm. The idea
>>>>>>>>>>>>         > >>>> is bullet proof and it must be
>>>>>>>>>>>>         > >>>> an implementation mess up. I would like to clarify
>>>>> before
>>>>>>>>>>>>         we draw a
>>>>>>>>>>>>         > >>>> conclusion.
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>    Repartitioning the scattered events back to
>> their
>>>>>>>>>>>>
>>>>>>>>>>> original
>>>>>>>>>
>>>>>>>>>>         > >>>>> partitions is the only way I know how to
>> conclusively
>>>>> deal
>>>>>>>>>>>>         with
>>>>>>>>>>>>         > >>>>> out-of-order events in a given time frame, and to
>>>>> ensure
>>>>>>>>>>>>         that the
>>>>>>>>>>>>         > data
>>>>>>>>>>>>         > >>>>> is
>>>>>>>>>>>>         > >>>>> eventually consistent with the input events.
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>> If you have some code to share that illustrates
>> your
>>>>>>>>>>>>         approach, I
>>>>>>>>>>>>         > would
>>>>>>>>>>>>         > >>>>> be
>>>>>>>>>>>>         > >>>>> very grateful as it would remove any
>>>>> misunderstandings
>>>>>>>>>>>>         that I may
>>>>>>>>>>>>         > have.
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>> ah okay you were looking for my code. I don't have
>>>>>>>>>>>>         something easily
>>>>>>>>>>>>         > >>>> readable here as its bloated with OO-patterns.
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>> its anyhow trivial:
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>> @Override
>>>>>>>>>>>>         > >>>>      public T apply(K aggKey, V value, T
>> aggregate)
>>>>>>>>>>>>         > >>>>      {
>>>>>>>>>>>>         > >>>>          Map<U, V> currentStateAsMap =
>>>>> asMap(aggregate);
>>>>>>>>>>>> <<
>>>>>>>>>>>>         imaginary
>>>>>>>>>>>>         > >>>>          U toModifyKey = mapper.apply(value);
>>>>>>>>>>>>         > >>>>              << this is the place where people
>>>>> actually
>>>>>>>>>>>>         gonna have
>>>>>>>>>>>>         > issues
>>>>>>>>>>>>         > >>>> and why you probably couldn't do it. we would need
>>>>> to find
>>>>>>>>>>>>         a solution
>>>>>>>>>>>>         > here.
>>>>>>>>>>>>         > >>>> I didn't realize that yet.
>>>>>>>>>>>>         > >>>>              << we propagate the field in the
>>>>> joiner, so
>>>>>>>>>>>>         that we can
>>>>>>>>>>>>         > pick
>>>>>>>>>>>>         > >>>> it up in an aggregate. Probably you have not
>> thought
>>>>> of
>>>>>>>>>>>>         this in your
>>>>>>>>>>>>         > >>>> approach right?
>>>>>>>>>>>>         > >>>>              << I am very open to find a generic
>>>>> solution
>>>>>>>>>>>>         here. In my
>>>>>>>>>>>>         > >>>> honest opinion this is broken in
>> KTableImpl.GroupBy
>>>>> that
>>>>>>>>>>>> it
>>>>>>>>>>>>         looses
>>>>>>>>>>>>         > the keys
>>>>>>>>>>>>         > >>>> and only maintains the aggregate key.
>>>>>>>>>>>>         > >>>>              << I abstracted it away back then way
>>>>> before
>>>>>>>>>>>> i
>>>>>>>>>>>> was
>>>>>>>>>>>>         > thinking
>>>>>>>>>>>>         > >>>> of oneToMany join. That is why I didn't realize
>> its
>>>>>>>>>>>>         significance here.
>>>>>>>>>>>>         > >>>>              << Opinions?
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>          for (V m : current)
>>>>>>>>>>>>         > >>>>          {
>>>>>>>>>>>>         > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>>>>>>>>         > >>>>          }
>>>>>>>>>>>>         > >>>>          if (isAdder)
>>>>>>>>>>>>         > >>>>          {
>>>>>>>>>>>>         > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>>>>>>>>         > >>>>          }
>>>>>>>>>>>>         > >>>>          else
>>>>>>>>>>>>         > >>>>          {
>>>>>>>>>>>>         > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>>>         > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>>>         > >>>>                  return null;
>>>>>>>>>>>>         > >>>>              }
>>>>>>>>>>>>         > >>>>          }
>>>>>>>>>>>>         > >>>>          retrun asAggregateType(currentStateAsMap)
>>>>>>>>>>>>         > >>>>      }
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>>
>>>>>>>>>>>>         > >>>> Thanks,
>>>>>>>>>>>>         > >>>>> Adam
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>>>>>>>>>>         > Jan.Filipiak@trivago.com <mailto:
>> Jan.Filipiak@trivago.com
>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>         > >>>>> wrote:
>>>>>>>>>>>>         > >>>>>
>>>>>>>>>>>>         > >>>>> Thanks Adam for bringing Matthias to speed!
>>>>>>>>>>>>         > >>>>>> about the differences. I think re-keying back
>>>>> should be
>>>>>>>>>>>>         optional at
>>>>>>>>>>>>         > >>>>>> best.
>>>>>>>>>>>>         > >>>>>> I would say we return a KScatteredTable with
>>>>> reshuffle()
>>>>>>>>>>>>         returning
>>>>>>>>>>>>         > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>>>>>>>>>>         repartitioning
>>>>>>>>>>>>         > >>>>>> optional.
>>>>>>>>>>>>         > >>>>>> I am also in a big favour of doing the out of
>> order
>>>>>>>>>>>>         processing using
>>>>>>>>>>>>         > >>>>>> group
>>>>>>>>>>>>         > >>>>>> by instead high water mark tracking.
>>>>>>>>>>>>         > >>>>>> Just because unbounded growth is just scary + It
>>>>> saves
>>>>>>>>>>>> us
>>>>>>>>>>>>         the header
>>>>>>>>>>>>         > >>>>>> stuff.
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>> I think the abstraction of always repartitioning
>>>>> back is
>>>>>>>>>>>>         just not so
>>>>>>>>>>>>         > >>>>>> strong. Like the work has been done before we
>>>>> partition
>>>>>>>>>>>>         back and
>>>>>>>>>>>>         > >>>>>> grouping
>>>>>>>>>>>>         > >>>>>> by something else afterwards is really common.
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>>>>>>>>         > >>>>>>
>>>>>>>>>>>>         > >>>>>> Hi Matthias
>>>>>>>>>>>>         > >>>>>>> Thank you for your feedback, I do appreciate
>> it!
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> While name spacing would be possible, it would
>>>>> require
>>>>>>>>>>>> to
>>>>>>>>>>>>         > deserialize
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime overhead.
>> I
>>>>> would
>>>>>>>>>>>>         suggest to
>>>>>>>>>>>>         > no
>>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead. If
>> this
>>>>>>>>>>>>
>>>>>>>>>>> becomes a
>>>>>>>>>
>>>>>>>>>>         > problem in
>>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
>> later
>>>>> on.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Agreed. I will go with using a reserved string
>>>>> and
>>>>>>>>>>>>         document it.
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> My main concern about the design it the type of
>>>>> the
>>>>>>>>>>>>         result KTable:
>>>>>>>>>>>>         > If
>>>>>>>>>>>>         > >>>>>>> I
>>>>>>>>>>>>         > >>>>>>> understood the proposal correctly,
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> In your example, you have table1 and table2
>>>>> swapped.
>>>>>>>>>>>>         Here is how it
>>>>>>>>>>>>         > >>>>>>> works
>>>>>>>>>>>>         > >>>>>>> currently:
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> 1) table1 has the records that contain the
>>>>> foreign key
>>>>>>>>>>>>         within their
>>>>>>>>>>>>         > >>>>>>> value.
>>>>>>>>>>>>         > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>>>         > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>>>         > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> 2) A Value mapper is required to extract the
>>>>> foreign
>>>>>>>>>>>> key.
>>>>>>>>>>>>         > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>>>>>>>>>>         <http://value.fk> )
>>>>>>>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> The mapper is applied to each element in
>> table1,
>>>>> and a
>>>>>>>>>>>>         new combined
>>>>>>>>>>>>         > >>>>>>> key is
>>>>>>>>>>>>         > >>>>>>> made:
>>>>>>>>>>>>         > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>>>         <B-c,
>>>>>>>>>>>>         > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> 3) The rekeyed events are copartitioned with
>>>>> table2:
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>>>>>>>         (fk=A,bar=2)>
>>>>>>>>>>>>         > >>>>>>> Table2: <A,X>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>>>>>>>         > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>>>>>>>         > >>>>>>> Table2: <B,Y>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> 4) From here, they can be joined together
>> locally
>>>>> by
>>>>>>>>>>>>         applying the
>>>>>>>>>>>>         > >>>>>>> joiner
>>>>>>>>>>>>         > >>>>>>> function.
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> At this point, Jan's design and my design
>>>>> deviate. My
>>>>>>>>>>>>         design goes
>>>>>>>>>>>>         > on
>>>>>>>>>>>>         > >>>>>>> to
>>>>>>>>>>>>         > >>>>>>> repartition the data post-join and resolve
>>>>> out-of-order
>>>>>>>>>>>>         arrival of
>>>>>>>>>>>>         > >>>>>>> records,
>>>>>>>>>>>>         > >>>>>>> finally returning the data keyed just the
>>>>> original key.
>>>>>>>>>>>>         I do not
>>>>>>>>>>>>         > >>>>>>> expose
>>>>>>>>>>>>         > >>>>>>> the
>>>>>>>>>>>>         > >>>>>>> CombinedKey or any of the internals outside of
>> the
>>>>>>>>>>>>         joinOnForeignKey
>>>>>>>>>>>>         > >>>>>>> function. This does make for larger footprint,
>>>>> but it
>>>>>>>>>>>>         removes all
>>>>>>>>>>>>         > >>>>>>> agency
>>>>>>>>>>>>         > >>>>>>> for resolving out-of-order arrivals and
>> handling
>>>>>>>>>>>>         CombinedKeys from
>>>>>>>>>>>>         > the
>>>>>>>>>>>>         > >>>>>>> user. I believe that this makes the function
>> much
>>>>>>>>>>>> easier
>>>>>>>>>>>>         to use.
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> Let me know if this helps resolve your
>> questions,
>>>>> and
>>>>>>>>>>>>         please feel
>>>>>>>>>>>>         > >>>>>>> free to
>>>>>>>>>>>>         > >>>>>>> add anything else on your mind.
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> Thanks again,
>>>>>>>>>>>>         > >>>>>>> Adam
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
>> Sax <
>>>>>>>>>>>>         > >>>>>>> matthias@confluent.io <mailto:
>>>>> matthias@confluent.io>>
>>>>>>>>>>>>
>>>>>>>>>>>>         > >>>>>>> wrote:
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>> Hi,
>>>>>>>>>>>>         > >>>>>>>
>>>>>>>>>>>>         > >>>>>>>> I am just catching up on this thread. I did
>> not
>>>>> read
>>>>>>>>>>>>         everything so
>>>>>>>>>>>>         > >>>>>>>> far,
>>>>>>>>>>>>         > >>>>>>>> but want to share couple of initial thoughts:
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Headers: I think there is a fundamental
>>>>> difference
>>>>>>>>>>>>         between header
>>>>>>>>>>>>         > >>>>>>>> usage
>>>>>>>>>>>>         > >>>>>>>> in this KIP and KP-258. For 258, we add
>> headers
>>>>> to
>>>>>>>>>>>>         changelog topic
>>>>>>>>>>>>         > >>>>>>>> that
>>>>>>>>>>>>         > >>>>>>>> are owned by Kafka Streams and nobody else is
>>>>> supposed
>>>>>>>>>>>>         to write
>>>>>>>>>>>>         > into
>>>>>>>>>>>>         > >>>>>>>> them. In fact, no user header are written into
>>>>> the
>>>>>>>>>>>>         changelog topic
>>>>>>>>>>>>         > >>>>>>>> and
>>>>>>>>>>>>         > >>>>>>>> thus, there are not conflicts.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Nevertheless, I don't see a big issue with
>> using
>>>>>>>>>>>>         headers within
>>>>>>>>>>>>         > >>>>>>>> Streams.
>>>>>>>>>>>>         > >>>>>>>> As long as we document it, we can have some
>>>>> "reserved"
>>>>>>>>>>>>         header keys
>>>>>>>>>>>>         > >>>>>>>> and
>>>>>>>>>>>>         > >>>>>>>> users are not allowed to use when processing
>>>>> data with
>>>>>>>>>>>>         Kafka
>>>>>>>>>>>>         > Streams.
>>>>>>>>>>>>         > >>>>>>>> IMHO, this should be ok.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> I think there is a safe way to avoid
>> conflicts,
>>>>> since
>>>>>>>>>>>> these
>>>>>>>>>>>>         > headers
>>>>>>>>>>>>         > >>>>>>>> are
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>> only needed in internal topics (I think):
>>>>>>>>>>>>         > >>>>>>>>> For internal and changelog topics, we can
>>>>> namespace
>>>>>>>>>>>>         all headers:
>>>>>>>>>>>>         > >>>>>>>>> * user-defined headers are namespaced as
>>>>> "external."
>>>>>>>>>>>> +
>>>>>>>>>>>>         headerKey
>>>>>>>>>>>>         > >>>>>>>>> * internal headers are namespaced as
>>>>> "internal." +
>>>>>>>>>>>>         headerKey
>>>>>>>>>>>>         > >>>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>> While name spacing would be possible, it
>> would
>>>>>>>>>>>> require
>>>>>>>>>>>>
>>>>>>>>>>> to
>>>>>>>>>
>>>>>>>>>>         > >>>>>>>> deserialize
>>>>>>>>>>>>         > >>>>>>>> user headers what implies a runtime overhead.
>> I
>>>>> would
>>>>>>>>>>>>         suggest to
>>>>>>>>>>>>         > no
>>>>>>>>>>>>         > >>>>>>>> namespace for now to avoid the overhead. If
>> this
>>>>>>>>>>>>
>>>>>>>>>>> becomes a
>>>>>>>>>
>>>>>>>>>>         > problem in
>>>>>>>>>>>>         > >>>>>>>> the future, we can still add name spacing
>> later
>>>>> on.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> My main concern about the design it the type
>> of
>>>>> the
>>>>>>>>>>>>         result KTable:
>>>>>>>>>>>>         > >>>>>>>> If I
>>>>>>>>>>>>         > >>>>>>>> understood the proposal correctly,
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>>>>>>         > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> KTable<K1,V3> joinedTable =
>>>>> table1.join(table2,...);
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> implies that the `joinedTable` has the same
>> key
>>>>> as the
>>>>>>>>>>>>         left input
>>>>>>>>>>>>         > >>>>>>>> table.
>>>>>>>>>>>>         > >>>>>>>> IMHO, this does not work because if table2
>>>>> contains
>>>>>>>>>>>>         multiple rows
>>>>>>>>>>>>         > >>>>>>>> that
>>>>>>>>>>>>         > >>>>>>>> join with a record in table1 (what is the main
>>>>> purpose
>>>>>>>>>>>>
>>>>>>>>>>> of
>>>>>>>>>
>>>>>>>>>> a
>>>>>>>>>>>>         > foreign
>>>>>>>>>>>>         > >>>>>>>> key
>>>>>>>>>>>>         > >>>>>>>> join), the result table would only contain a
>>>>> single
>>>>>>>>>>>>         join result,
>>>>>>>>>>>>         > but
>>>>>>>>>>>>         > >>>>>>>> not
>>>>>>>>>>>>         > >>>>>>>> multiple.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Example:
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> table1 input stream: <A,X>
>>>>>>>>>>>>         > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> We use table2 value a foreign key to table1
>> key
>>>>> (ie,
>>>>>>>>>>>>         "A" joins).
>>>>>>>>>>>>         > If
>>>>>>>>>>>>         > >>>>>>>> the
>>>>>>>>>>>>         > >>>>>>>> result key is the same key as key of table1,
>> this
>>>>>>>>>>>>         implies that the
>>>>>>>>>>>>         > >>>>>>>> result can either be <A, join(X,1)> or <A,
>>>>> join(X,2)>
>>>>>>>>>>>>         but not
>>>>>>>>>>>>         > both.
>>>>>>>>>>>>         > >>>>>>>> Because the share the same key, whatever
>> result
>>>>> record
>>>>>>>>>>>>         we emit
>>>>>>>>>>>>         > later,
>>>>>>>>>>>>         > >>>>>>>> overwrite the previous result.
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> This is the reason why Jan originally proposed
>>>>> to use
>>>>>>>>>>>> a
>>>>>>>>>>>>         > combination
>>>>>>>>>>>>         > >>>>>>>> of
>>>>>>>>>>>>         > >>>>>>>> both primary keys of the input tables as key
>> of
>>>>> the
>>>>>>>>>>>>         output table.
>>>>>>>>>>>>         > >>>>>>>> This
>>>>>>>>>>>>         > >>>>>>>> makes the keys of the output table unique and
>> we
>>>>> can
>>>>>>>>>>>>         store both in
>>>>>>>>>>>>         > >>>>>>>> the
>>>>>>>>>>>>         > >>>>>>>> output table:
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>>>>> join(X,2)>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Thoughts?
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> -Matthias
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>>>>>>         > >>>>>>>>
>>>>>>>>>>>>         > >>>>>>>> Just on remark here.
>>>>>>>>>>>>         > >>>>>>>>> The high-watermark could be disregarded. The
>>>>> decision
>>>>>>>>>>>>         about the
>>>>>>>>>>>>         > >>>>>>>>> forward
>>>>>>>>>>>>         > >>>>>>>>> depends on the size of the aggregated map.
>>>>>>>>>>>>         > >>>>>>>>> Only 1 element long maps would be unpacked
>> and
>>>>>>>>>>>>         forwarded. 0
>>>>>>>>>>>>         > element
>>>>>>>>>>>>         > >>>>>>>>> maps
>>>>>>>>>>>>         > >>>>>>>>> would be published as delete. Any other count
>>>>>>>>>>>>         > >>>>>>>>> of map entries is in "waiting for correct
>>>>> deletes to
>>>>>>>>>>>>         > arrive"-state.
>>>>>>>>>>>>         > >>>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>>>>>>         > >>>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>> It does look like I could replace the second
>>>>>>>>>>>>         repartition store
>>>>>>>>>>>>         > and
>>>>>>>>>>>>         > >>>>>>>>>> highwater store with a groupBy and reduce.
>>>>> However,
>>>>>>>>>>>>         it looks
>>>>>>>>>>>>         > like
>>>>>>>>>>>>         > >>>>>>>>>> I
>>>>>>>>>>>>         > >>>>>>>>>> would
>>>>>>>>>>>>         > >>>>>>>>>> still need to store the highwater value
>> within
>>>>> the
>>>>>>>>>>>>         materialized
>>>>>>>>>>>>         > >>>>>>>>>> store,
>>>>>>>>>>>>         > >>>>>>>>>>
>>>>>>>>>>>>         > >>>>>>>>>> to
>>>>>>>>>>>>         > >>>>>>>>> compare the arrival of out-of-order records
>>>>> (assuming
>>>>>>>>>>>>
>>>>>>>>>>> my
>>>>>>>>>
>>>>>>>>>>         > >>>>>>>>> understanding
>>>>>>>>>>>>         > >>>>>>>>> of
>>>>>>>>>>>>         > >>>>>>>>> THIS is correct...). This in effect is the
>> same
>>>>> as
>>>>>>>>>>>> the
>>>>>>>>>>>>         design I
>>>>>>>>>>>>         > have
>>>>>>>>>>>>         > >>>>>>>>> now,
>>>>>>>>>>>>         > >>>>>>>>> just with the two tables merged together.
>>>>>>>>>>>>         > >>>>>>>>>
>>>>>>>>>>>>         >
>>>>>>>>>>>>         >
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> -- Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi all,

Sorry that this discussion petered out... I think the 2.1 release caused an
extended distraction that pushed it off everyone's radar (which was
precisely Adam's concern). Personally, I've also had some extend
distractions of my own that kept (and continue to keep) me preoccupied.

However, calling for a vote did wake me up, so I guess Jan was on the right
track!

I've gone back and reviewed the whole KIP document and the prior
discussion, and I'd like to offer a few thoughts:

API Thoughts:

1. If I read the KIP right, you are proposing a many-to-one join. Could we
consider naming it manyToOneJoin? Or, if you prefer, flip the design around
and make it a oneToManyJoin?

The proposed name "joinOnForeignKey" disguises the join type, and it seems
like it might trick some people into using it for a one-to-one join. This
would work, of course, but it would be super inefficient compared to a
simple rekey-and-join.

2. I might have missed it, but I don't think it's specified whether it's an
inner, outer, or left join. I'm guessing an outer join, as (neglecting IQ),
the rest can be achieved by filtering or by handling it in the ValueJoiner.

3. The arg list to joinOnForeignKey doesn't look quite right.
3a. Regarding Serialized: There are a few different paradigms in play in
the Streams API, so it's confusing, but instead of three Serialized args, I
think it would be better to have one that allows (optionally) setting the 4
incoming serdes. The result serde is defined by the Materialized. The
incoming serdes can be optional because they might already be available on
the source KTables, or the default serdes from the config might be
applicable.

3b. Is the StreamPartitioner necessary? The other joins don't allow setting
one, and it seems like it might actually be harmful, since the rekey
operation needs to produce results that are co-partitioned with the "other"
KTable.

4. I'm fine with the "reserved word" header, but I didn't actually follow
what Matthias meant about namespacing requiring "deserializing" the record
header. The headers are already Strings, so I don't think that
deserialization is required. If we applied the namespace at source nodes
and stripped it at sink nodes, this would be practically no overhead. The
advantage of the namespace idea is that no public API change wrt headers
needs to happen, and no restrictions need to be placed on users' headers.

(Although I'm wondering if we can get away without the header at all...
stay tuned)

5. I also didn't follow the discussion about the HWM table growing without
bound. As I read it, the HWM table is effectively implementing OCC to
resolve the problem you noted with disordering when the rekey is
reversed... particularly notable when the FK changes. As such, it only
needs to track the most recent "version" (the offset in the source
partition) of each key. Therefore, it should have the same number of keys
as the source table at all times.

I see that you are aware of KIP-258, which I think might be relevant in a
couple of ways. One: it's just about storing the timestamp in the state
store, but the ultimate idea is to effectively use the timestamp as an OCC
"version" to drop disordered updates. You wouldn't want to use the
timestamp for this operation, but if you were to use a similar mechanism to
store the source offset in the store alongside the re-keyed values, then
you could avoid a separate table.

6. You and Jan have been thinking about this for a long time, so I've
probably missed something here, but I'm wondering if we can avoid the HWM
tracking at all and resolve out-of-order during a final join instead...

Let's say we're joining a left table (Integer K: Letter FK, (other data))
to a right table (Letter K: (some data)).

Left table:
1: (A, xyz)
2: (B, asd)

Right table:
A: EntityA
B: EntityB

We could do a rekey as you proposed with a combined key, but not
propagating the value at all..
Rekey table:
A-1: (dummy value)
B-2: (dummy value)

Which we then join with the right table to produce:
A-1: EntityA
B-2: EntityB

Which gets rekeyed back:
1: A, EntityA
2: B, EntityB

And finally we do the actual join:
Result table:
1: ((A, xyz), EntityA)
2: ((B, asd), EntityB)

The thing is that in that last join, we have the opportunity to compare the
current FK in the left table with the incoming PK of the right table. If
they don't match, we just drop the event, since it must be outdated.

In your KIP, you gave an example in which (1: A, xyz) gets updated to (1:
B, xyz), ultimately yielding a conundrum about whether the final state
should be (1: null) or (1: joined-on-B). With the algorithm above, you
would be considering (1: (B, xyz), (A, null)) vs (1: (B, xyz), (B,
EntityB)). It seems like this does give you enough information to make the
right choice, regardless of disordering.


7. Last thought... I'm a little concerned about the performance of the
range scans when records change in the right table. You've said that you've
been using the algorithm you presented in production for a while. Can you
give us a sense of the performance characteristics you've observed?

I could only think of one alternative, but I'm not sure if it's better or
worse... If the first re-key only needs to preserve the original key, as I
proposed in #6, then we could store a vector of keys in the value:

Left table:
1: A,...
2: B,...
3: A,...

Gets re-keyed:
A: [1, 3]
B: [2]

Then, the rhs part of the join would only need a regular single-key lookup.
Of course we have to deal with the problem of large values, as there's no
bound on the number of lhs records that can reference rhs records. Offhand,
I'd say we could page the values, so when one row is past the threshold, we
append the key for the next page. Then in most cases, it would be a single
key lookup, but for large fan-out updates, it would be one per (max value
size)/(avg lhs key size).

This seems more complex, though... Plus, I think there's some extra
tracking we'd need to do to know when to emit a retraction. For example,
when record 1 is deleted, the re-key table would just have (A: [3]). Some
kind of tombstone is needed so that the join result for 1 can also be
retracted.

That's all!

Thanks so much to both Adam and Jan for the thoughtful KIP. Sorry the
discussion has been slow.
-John


On Fri, Oct 12, 2018 at 2:20 AM Jan Filipiak <Ja...@trivago.com>
wrote:

> Id say you can just call the vote.
>
> that happens all the time, and if something comes up, it just goes back
> to discuss.
>
> would not expect to much attention with another another email in this
> thread.
>
> best Jan
>
> On 09.10.2018 13:56, Adam Bellemare wrote:
> > Hello Contributors
> >
> > I know that 2.1 is about to be released, but I do need to bump this to
> keep
> > visibility up. I am still intending to push this through once contributor
> > feedback is given.
> >
> > Main points that need addressing:
> > 1) Any way (or benefit) in structuring the current singular graph node
> into
> > multiple nodes? It has a whopping 25 parameters right now. I am a bit
> fuzzy
> > on how the optimizations are supposed to work, so I would appreciate any
> > help on this aspect.
> >
> > 2) Overall strategy for joining + resolving. This thread has much
> discourse
> > between Jan and I between the current highwater mark proposal and a
> groupBy
> > + reduce proposal. I am of the opinion that we need to strictly handle
> any
> > chance of out-of-order data and leave none of it up to the consumer. Any
> > comments or suggestions here would also help.
> >
> > 3) Anything else that you see that would prevent this from moving to a
> vote?
> >
> > Thanks
> >
> > Adam
> >
> >
> >
> >
> >
> >
> >
> > On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <
> adam.bellemare@gmail.com>
> > wrote:
> >
> >> Hi Jan
> >>
> >> With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you
> >> actually only need to specify the amount of segments you want and how
> large
> >> they are. To the best of my understanding, what happens is that the
> >> segments are automatically rolled over as new data with new timestamps
> are
> >> created. We use this exact functionality in some of the work done
> >> internally at my company. For reference, this is the hopping windowed
> store.
> >>
> >>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
> >>
> >> In the code that I have provided, there are going to be two 24h
> segments.
> >> When a record is put into the windowStore, it will be inserted at time
> T in
> >> both segments. The two segments will always overlap by 12h. As time
> goes on
> >> and new records are added (say at time T+12h+), the oldest segment will
> be
> >> automatically deleted and a new segment created. The records are by
> default
> >> inserted with the context.timestamp(), such that it is the record time,
> not
> >> the clock time, which is used.
> >>
> >> To the best of my understanding, the timestamps are retained when
> >> restoring from the changelog.
> >>
> >> Basically, this is heavy-handed way to deal with TTL at a segment-level,
> >> instead of at an individual record level.
> >>
> >> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Ja...@trivago.com>
> >> wrote:
> >>
> >>> Will that work? I expected it to blow up with ClassCastException or
> >>> similar.
> >>>
> >>> You either would have to specify the window you fetch/put or iterate
> >>> across all windows the key was found in right?
> >>>
> >>> I just hope the window-store doesn't check stream-time under the hoods
> >>> that would be a questionable interface.
> >>>
> >>> If it does: did you see my comment on checking all the windows earlier?
> >>> that would be needed to actually give reasonable time gurantees.
> >>>
> >>> Best
> >>>
> >>>
> >>>
> >>> On 25.09.2018 13:18, Adam Bellemare wrote:
> >>>> Hi Jan
> >>>>
> >>>> Check for  " highwaterMat " in the PR. I only changed the state store,
> >>> not
> >>>> the ProcessorSupplier.
> >>>>
> >>>> Thanks,
> >>>> Adam
> >>>>
> >>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <
> Jan.Filipiak@trivago.com
> >>>>
> >>>> wrote:
> >>>>
> >>>>>
> >>>>>
> >>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>>>>
> >>>>>> @Guozhang
> >>>>>>
> >>>>>> Thanks for the information. This is indeed something that will be
> >>>>>> extremely
> >>>>>> useful for this KIP.
> >>>>>>
> >>>>>> @Jan
> >>>>>> Thanks for your explanations. That being said, I will not be moving
> >>> ahead
> >>>>>> with an implementation using reshuffle/groupBy solution as you
> >>> propose.
> >>>>>> That being said, if you wish to implement it yourself off of my
> >>> current PR
> >>>>>> and submit it as a competitive alternative, I would be more than
> >>> happy to
> >>>>>> help vet that as an alternate solution. As it stands right now, I do
> >>> not
> >>>>>> really have more time to invest into alternatives without there
> being
> >>> a
> >>>>>> strong indication from the binding voters which they would prefer.
> >>>>>>
> >>>>>>
> >>>>> Hey, total no worries. I think I personally gave up on the streams
> DSL
> >>> for
> >>>>> some time already, otherwise I would have pulled this KIP through
> >>> already.
> >>>>> I am currently reimplementing my own DSL based on PAPI.
> >>>>>
> >>>>>
> >>>>>> I will look at finishing up my PR with the windowed state store in
> the
> >>>>>> next
> >>>>>> week or so, exercising it via tests, and then I will come back for
> >>> final
> >>>>>> discussions. In the meantime, I hope that any of the binding voters
> >>> could
> >>>>>> take a look at the KIP in the wiki. I have updated it according to
> the
> >>>>>> latest plan:
> >>>>>>
> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>>>>> Support+non-key+joining+in+KTable
> >>>>>>
> >>>>>> I have also updated the KIP PR to use a windowed store. This could
> be
> >>>>>> replaced by the results of KIP-258 whenever they are completed.
> >>>>>> https://github.com/apache/kafka/pull/5527
> >>>>>>
> >>>>>> Thanks,
> >>>>>>
> >>>>>> Adam
> >>>>>>
> >>>>>
> >>>>> Is the HighWatermarkResolverProccessorsupplier already updated in the
> >>> PR?
> >>>>> expected it to change to Windowed<K>,Long Missing something?
> >>>>>
> >>>>>
> >>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as it
> >>> is
> >>>>>>> for
> >>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
> >>> want to
> >>>>>>> have "handling out-of-order data for source KTable" such that
> >>> instead of
> >>>>>>> blindly apply the updates to the materialized store, i.e. following
> >>>>>>> offset
> >>>>>>> ordering, we will reject updates that are older than the current
> >>> key's
> >>>>>>> timestamps, i.e. following timestamp ordering.
> >>>>>>>
> >>>>>>>
> >>>>>>> Guozhang
> >>>>>>>
> >>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <
> wangguoz@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>> Hello Adam,
> >>>>>>>>
> >>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the
> high
> >>>>>>>> watermark store, now altered to be replaced with a window store),
> I
> >>>>>>>> think
> >>>>>>>> another current on-going KIP may actually help:
> >>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> This is for adding the timestamp into a key-value store (i.e. only
> >>> for
> >>>>>>>> non-windowed KTable), and then one of its usage, as described in
> >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
> >>> then
> >>>>>>>> "reject" updates from the source topics if its timestamp is
> smaller
> >>> than
> >>>>>>>> the current key's latest update timestamp. I think it is very
> >>> similar to
> >>>>>>>> what you have in mind for high watermark based filtering, while
> you
> >>> only
> >>>>>>>> need to make sure that the timestamps of the joining records are
> >>>>>>>>
> >>>>>>> correctly
> >>>>>>>
> >>>>>>>> inherited though the whole topology to the final stage.
> >>>>>>>>
> >>>>>>>> Note that this KIP is for key-value store and hence non-windowed
> >>> KTables
> >>>>>>>> only, but for windowed KTables we do not really have a good
> support
> >>> for
> >>>>>>>> their joins anyways (
> >>> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>>>>> I
> >>>>>>>> think we can just consider non-windowed KTable-KTable non-key
> joins
> >>> for
> >>>>>>>> now. In which case, KIP-258 should help.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> >>> Jan.Filipiak@trivago.com
> >>>>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Guozhang
> >>>>>>>>>>
> >>>>>>>>>> Current highwater mark implementation would grow endlessly based
> >>> on
> >>>>>>>>>> primary key of original event. It is a pair of (<this table
> >>> primary
> >>>>>>>>>>
> >>>>>>>>> key>,
> >>>>>>>
> >>>>>>>> <highest offset seen for that key>). This is used to differentiate
> >>>>>>>>>>
> >>>>>>>>> between
> >>>>>>>
> >>>>>>>> late arrivals and new updates. My newest proposal would be to
> >>> replace
> >>>>>>>>>>
> >>>>>>>>> it
> >>>>>>>
> >>>>>>>> with a Windowed state store of Duration N. This would allow the
> same
> >>>>>>>>>> behaviour, but cap the size based on time. This should allow for
> >>> all
> >>>>>>>>>> late-arriving events to be processed, and should be customizable
> >>> by
> >>>>>>>>>> the
> >>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes
> of
> >>>>>>>>>>
> >>>>>>>>> window,
> >>>>>>>
> >>>>>>>> or perhaps 7 days...).
> >>>>>>>>>>
> >>>>>>>>>> Hi Adam, using time based retention can do the trick here. Even
> >>> if I
> >>>>>>>>> would still like to see the automatic repartitioning optional
> >>> since I
> >>>>>>>>>
> >>>>>>>> would
> >>>>>>>
> >>>>>>>> just reshuffle again. With windowed store I am a little bit
> >>> sceptical
> >>>>>>>>>
> >>>>>>>> about
> >>>>>>>
> >>>>>>>> how to determine the window. So esentially one could run into
> >>> problems
> >>>>>>>>>
> >>>>>>>> when
> >>>>>>>
> >>>>>>>> the rapid change happens near a window border. I will check you
> >>>>>>>>> implementation in detail, if its problematic, we could still
> check
> >>>>>>>>> _all_
> >>>>>>>>> windows on read with not to bad performance impact I guess. Will
> >>> let
> >>>>>>>>> you
> >>>>>>>>> know if the implementation would be correct as is. I wouldn't not
> >>> like
> >>>>>>>>>
> >>>>>>>> to
> >>>>>>>
> >>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  <
> timestamp(B).
> >>> I
> >>>>>>>>>
> >>>>>>>> think
> >>>>>>>
> >>>>>>>> we can't expect that.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> @Jan
> >>>>>>>>>> I believe I understand what you mean now - thanks for the
> >>> diagram, it
> >>>>>>>>>> did really help. You are correct that I do not have the original
> >>>>>>>>>>
> >>>>>>>>> primary
> >>>>>>>
> >>>>>>>> key available, and I can see that if it was available then you
> >>> would be
> >>>>>>>>>> able to add and remove events from the Map. That being said, I
> >>>>>>>>>>
> >>>>>>>>> encourage
> >>>>>>>
> >>>>>>>> you to finish your diagrams / charts just for clarity for everyone
> >>>>>>>>>>
> >>>>>>>>> else.
> >>>>>>>
> >>>>>>>>
> >>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> >>> understand
> >>>>>>>>>>
> >>>>>>>>> the benefits for the rest. Sorry about the original primary key,
> We
> >>>>>>>>> have
> >>>>>>>>> join and Group by implemented our own in PAPI and basically not
> >>> using
> >>>>>>>>>
> >>>>>>>> any
> >>>>>>>
> >>>>>>>> DSL (Just the abstraction). Completely missed that in original DSL
> >>> its
> >>>>>>>>>
> >>>>>>>> not
> >>>>>>>
> >>>>>>>> there and just assumed it. total brain mess up on my end. Will
> >>> finish
> >>>>>>>>>
> >>>>>>>> the
> >>>>>>>
> >>>>>>>> chart as soon as i get a quite evening this week.
> >>>>>>>>>
> >>>>>>>>> My follow up question for you is, won't the Map stay inside the
> >>> State
> >>>>>>>>>
> >>>>>>>>>> Store indefinitely after all of the changes have propagated?
> Isn't
> >>>>>>>>>> this
> >>>>>>>>>> effectively the same as a highwater mark state store?
> >>>>>>>>>>
> >>>>>>>>>> Thing is that if the map is empty, substractor is gonna return
> >>> `null`
> >>>>>>>>>
> >>>>>>>> and
> >>>>>>>
> >>>>>>>> the key is removed from the keyspace. But there is going to be a
> >>> store
> >>>>>>>>> 100%, the good thing is that I can use this store directly for
> >>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
> >>> satisfying
> >>>>>>>>> all gurantees needed for further groupby / join. The Windowed
> >>> store is
> >>>>>>>>>
> >>>>>>>> not
> >>>>>>>
> >>>>>>>> keeping the values, so for the next statefull operation we would
> >>>>>>>>> need to instantiate an extra store. or we have the window store
> >>> also
> >>>>>>>>>
> >>>>>>>> have
> >>>>>>>
> >>>>>>>> the values then.
> >>>>>>>>>
> >>>>>>>>> Long story short. if we can flip in a custom group by before
> >>>>>>>>> repartitioning to the original primary key i think it would help
> >>> the
> >>>>>>>>>
> >>>>>>>> users
> >>>>>>>
> >>>>>>>> big time in building efficient apps. Given the original primary
> key
> >>>>>>>>>
> >>>>>>>> issue I
> >>>>>>>
> >>>>>>>> understand that we do not have a solid foundation to build on.
> >>>>>>>>> Leaving primary key carry along to the user. very unfortunate. I
> >>> could
> >>>>>>>>> understand the decision goes like that. I do not think its a good
> >>>>>>>>>
> >>>>>>>> decision.
> >>>>>>>
> >>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks
> >>>>>>>>>> Adam
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>>>>>>>>> dumbreprajakta311@gmail.com <mailto:dumbreprajakta311@gmail.com
> >>
> >>>>>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>
> >>>>>>>>
> >>>>>>>>>>        please remove me from this group
> >>>>>>>>>>
> >>>>>>>>>>        On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> Jan.Filipiak@trivago.com
> >>>>>
> >>>>>>>>>>
> >>>>>>>>>>        wrote:
> >>>>>>>>>>
> >>>>>>>>>>        > Hi Adam,
> >>>>>>>>>>        >
> >>>>>>>>>>        > give me some time, will make such a chart. last time i
> >>> didn't
> >>>>>>>>>>        get along
> >>>>>>>>>>        > well with giphy and ruined all your charts.
> >>>>>>>>>>        > Hopefully i can get it done today
> >>>>>>>>>>        >
> >>>>>>>>>>        > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>>>>>>>>        > > Hi Jan
> >>>>>>>>>>        > >
> >>>>>>>>>>        > > I have included a diagram of what I attempted on the
> >>> KIP.
> >>>>>>>>>>        > >
> >>>>>>>>>>        >
> >>>>>>>>>>
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>>>>        <
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>>>>        > >
> >>>>>>>>>>        > > I attempted this back at the start of my own
> >>> implementation
> >>>>>>>>>> of
> >>>>>>>>>>        this
> >>>>>>>>>>        > > solution, and since I could not get it to work I have
> >>> since
> >>>>>>>>>>        discarded the
> >>>>>>>>>>        > > code. At this point in time, if you wish to continue
> >>> pursuing
> >>>>>>>>>>        for your
> >>>>>>>>>>        > > groupBy solution, I ask that you please create a
> >>> diagram on
> >>>>>>>>>>        the KIP
> >>>>>>>>>>        > > carefully explaining your solution. Please feel free
> to
> >>> use
> >>>>>>>>>>        the image I
> >>>>>>>>>>        > > just posted as a starting point. I am having trouble
> >>>>>>>>>>        understanding your
> >>>>>>>>>>        > > explanations but I think that a carefully constructed
> >>> diagram
> >>>>>>>>>>        will clear
> >>>>>>>>>>        > up
> >>>>>>>>>>        > > any misunderstandings. Alternately, please post a
> >>>>>>>>>>        comprehensive PR with
> >>>>>>>>>>        > > your solution. I can only guess at what you mean, and
> >>> since I
> >>>>>>>>>>        value my
> >>>>>>>>>>        > own
> >>>>>>>>>>        > > time as much as you value yours, I believe it is your
> >>>>>>>>>>        responsibility to
> >>>>>>>>>>        > > provide an implementation instead of me trying to
> guess.
> >>>>>>>>>>        > >
> >>>>>>>>>>        > > Adam
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >
> >>>>>>>>>>        > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> Jan.Filipiak@trivago.com
> >>>>>
> >>>>>>>>>>
> >>>>>>>>>>        > > wrote:
> >>>>>>>>>>        > >
> >>>>>>>>>>        > >> Hi James,
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >> nice to see you beeing interested. kafka streams at
> >>> this
> >>>>>>>>>>        point supports
> >>>>>>>>>>        > >> all sorts of joins as long as both streams have the
> >>> same
> >>>>>>>>>> key.
> >>>>>>>>>>        > >> Adam is currently implementing a join where a KTable
> >>> and a
> >>>>>>>>>>        KTable can
> >>>>>>>>>>        > have
> >>>>>>>>>>        > >> a one to many relation ship (1:n). We exploit that
> >>> rocksdb
> >>>>>>>>>> is
> >>>>>>>>>>
> >>>>>>>>> a
> >>>>>>>
> >>>>>>>>        > >> datastore that keeps data sorted (At least exposes an
> >>> API to
> >>>>>>>>>>        access the
> >>>>>>>>>>        > >> stored data in a sorted fashion).
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >> I think the technical caveats are well understood
> now
> >>> and we
> >>>>>>>>>>
> >>>>>>>>> are
> >>>>>>>
> >>>>>>>>        > basically
> >>>>>>>>>>        > >> down to philosophy and API Design ( when Adam sees
> my
> >>> newest
> >>>>>>>>>>        message).
> >>>>>>>>>>        > >> I have a lengthy track record of loosing those kinda
> >>>>>>>>>>        arguments within
> >>>>>>>>>>        > the
> >>>>>>>>>>        > >> streams community and I have no clue why. So I
> >>> literally
> >>>>>>>>>>        can't wait for
> >>>>>>>>>>        > you
> >>>>>>>>>>        > >> to churn through this thread and give you opinion on
> >>> how we
> >>>>>>>>>>        should
> >>>>>>>>>>        > design
> >>>>>>>>>>        > >> the return type of the oneToManyJoin and how many
> >>> power we
> >>>>>>>>>>        want to give
> >>>>>>>>>>        > to
> >>>>>>>>>>        > >> the user vs "simplicity" (where simplicity isn't
> >>> really that
> >>>>>>>>>>        as users
> >>>>>>>>>>        > still
> >>>>>>>>>>        > >> need to understand it I argue)
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >> waiting for you to join in on the discussion
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >> Best Jan
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>>>>>>>>        > >>
> >>>>>>>>>>        > >>> I am new to this group and I found this subject
> >>>>>>>>>>        interesting.  Sounds
> >>>>>>>>>>        > like
> >>>>>>>>>>        > >>> you guys want to implement a join table of two
> >>> streams? Is
> >>>>>>>>>> there
> >>>>>>>>>>        > somewhere
> >>>>>>>>>>        > >>> I can see the original requirement or proposal?
> >>>>>>>>>>        > >>>
> >>>>>>>>>>        > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:
> Jan.Filipiak@trivago.com
> >>>>>
> >>>>>>>>>>
> >>>>>>>>>>        > >>>> wrote:
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>> I'm currently testing using a Windowed Store to
> >>> store the
> >>>>>>>>>>        highwater
> >>>>>>>>>>        > >>>>> mark.
> >>>>>>>>>>        > >>>>> By all indications this should work fine, with
> the
> >>> caveat
> >>>>>>>>>>        being that
> >>>>>>>>>>        > it
> >>>>>>>>>>        > >>>>> can
> >>>>>>>>>>        > >>>>> only resolve out-of-order arrival for up to the
> >>> size of
> >>>>>>>>>>        the window
> >>>>>>>>>>        > (ie:
> >>>>>>>>>>        > >>>>> 24h, 72h, etc). This would remove the possibility
> >>> of it
> >>>>>>>>>>
> >>>>>>>>> being
> >>>>>>>
> >>>>>>>>        > unbounded
> >>>>>>>>>>        > >>>>> in
> >>>>>>>>>>        > >>>>> size.
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>> With regards to Jan's suggestion, I believe this
> is
> >>> where
> >>>>>>>>>>        we will
> >>>>>>>>>>        > have
> >>>>>>>>>>        > >>>>> to
> >>>>>>>>>>        > >>>>> remain in disagreement. While I do not disagree
> >>> with your
> >>>>>>>>>>        statement
> >>>>>>>>>>        > >>>>> about
> >>>>>>>>>>        > >>>>> there likely to be additional joins done in a
> >>> real-world
> >>>>>>>>>>        workflow, I
> >>>>>>>>>>        > do
> >>>>>>>>>>        > >>>>> not
> >>>>>>>>>>        > >>>>> see how you can conclusively deal with
> out-of-order
> >>>>>>>>>> arrival
> >>>>>>>>>> of
> >>>>>>>>>>        > >>>>> foreign-key
> >>>>>>>>>>        > >>>>> changes and subsequent joins. I have attempted
> what
> >>> I
> >>>>>>>>>>        think you have
> >>>>>>>>>>        > >>>>> proposed (without a high-water, using groupBy and
> >>> reduce)
> >>>>>>>>>>        and found
> >>>>>>>>>>        > >>>>> that if
> >>>>>>>>>>        > >>>>> the foreign key changes too quickly, or the load
> on
> >>> a
> >>>>>>>>>>        stream thread
> >>>>>>>>>>        > is
> >>>>>>>>>>        > >>>>> too
> >>>>>>>>>>        > >>>>> high, the joined messages will arrive
> out-of-order
> >>> and be
> >>>>>>>>>>        incorrectly
> >>>>>>>>>>        > >>>>> propagated, such that an intermediate event is
> >>>>>>>>>> represented
> >>>>>>>>>>        as the
> >>>>>>>>>>        > final
> >>>>>>>>>>        > >>>>> event.
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>> Can you shed some light on your groupBy
> >>> implementation.
> >>>>>>>>>>        There must be
> >>>>>>>>>>        > >>>> some sort of flaw in it.
> >>>>>>>>>>        > >>>> I have a suspicion where it is, I would just like
> to
> >>>>>>>>>>        confirm. The idea
> >>>>>>>>>>        > >>>> is bullet proof and it must be
> >>>>>>>>>>        > >>>> an implementation mess up. I would like to clarify
> >>> before
> >>>>>>>>>>        we draw a
> >>>>>>>>>>        > >>>> conclusion.
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>    Repartitioning the scattered events back to
> their
> >>>>>>>>>>
> >>>>>>>>> original
> >>>>>>>
> >>>>>>>>        > >>>>> partitions is the only way I know how to
> conclusively
> >>> deal
> >>>>>>>>>>        with
> >>>>>>>>>>        > >>>>> out-of-order events in a given time frame, and to
> >>> ensure
> >>>>>>>>>>        that the
> >>>>>>>>>>        > data
> >>>>>>>>>>        > >>>>> is
> >>>>>>>>>>        > >>>>> eventually consistent with the input events.
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>> If you have some code to share that illustrates
> your
> >>>>>>>>>>        approach, I
> >>>>>>>>>>        > would
> >>>>>>>>>>        > >>>>> be
> >>>>>>>>>>        > >>>>> very grateful as it would remove any
> >>> misunderstandings
> >>>>>>>>>>        that I may
> >>>>>>>>>>        > have.
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>> ah okay you were looking for my code. I don't have
> >>>>>>>>>>        something easily
> >>>>>>>>>>        > >>>> readable here as its bloated with OO-patterns.
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>> its anyhow trivial:
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>> @Override
> >>>>>>>>>>        > >>>>      public T apply(K aggKey, V value, T
> aggregate)
> >>>>>>>>>>        > >>>>      {
> >>>>>>>>>>        > >>>>          Map<U, V> currentStateAsMap =
> >>> asMap(aggregate);
> >>>>>>>>>> <<
> >>>>>>>>>>        imaginary
> >>>>>>>>>>        > >>>>          U toModifyKey = mapper.apply(value);
> >>>>>>>>>>        > >>>>              << this is the place where people
> >>> actually
> >>>>>>>>>>        gonna have
> >>>>>>>>>>        > issues
> >>>>>>>>>>        > >>>> and why you probably couldn't do it. we would need
> >>> to find
> >>>>>>>>>>        a solution
> >>>>>>>>>>        > here.
> >>>>>>>>>>        > >>>> I didn't realize that yet.
> >>>>>>>>>>        > >>>>              << we propagate the field in the
> >>> joiner, so
> >>>>>>>>>>        that we can
> >>>>>>>>>>        > pick
> >>>>>>>>>>        > >>>> it up in an aggregate. Probably you have not
> thought
> >>> of
> >>>>>>>>>>        this in your
> >>>>>>>>>>        > >>>> approach right?
> >>>>>>>>>>        > >>>>              << I am very open to find a generic
> >>> solution
> >>>>>>>>>>        here. In my
> >>>>>>>>>>        > >>>> honest opinion this is broken in
> KTableImpl.GroupBy
> >>> that
> >>>>>>>>>> it
> >>>>>>>>>>        looses
> >>>>>>>>>>        > the keys
> >>>>>>>>>>        > >>>> and only maintains the aggregate key.
> >>>>>>>>>>        > >>>>              << I abstracted it away back then way
> >>> before
> >>>>>>>>>> i
> >>>>>>>>>> was
> >>>>>>>>>>        > thinking
> >>>>>>>>>>        > >>>> of oneToMany join. That is why I didn't realize
> its
> >>>>>>>>>>        significance here.
> >>>>>>>>>>        > >>>>              << Opinions?
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>          for (V m : current)
> >>>>>>>>>>        > >>>>          {
> >>>>>>>>>>        > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >>>>>>>>>>        > >>>>          }
> >>>>>>>>>>        > >>>>          if (isAdder)
> >>>>>>>>>>        > >>>>          {
> >>>>>>>>>>        > >>>> currentStateAsMap.put(toModifyKey, value);
> >>>>>>>>>>        > >>>>          }
> >>>>>>>>>>        > >>>>          else
> >>>>>>>>>>        > >>>>          {
> >>>>>>>>>>        > >>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>>>>        > >>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>>>>        > >>>>                  return null;
> >>>>>>>>>>        > >>>>              }
> >>>>>>>>>>        > >>>>          }
> >>>>>>>>>>        > >>>>          retrun asAggregateType(currentStateAsMap)
> >>>>>>>>>>        > >>>>      }
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>>
> >>>>>>>>>>        > >>>> Thanks,
> >>>>>>>>>>        > >>>>> Adam
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> >>>>>>>>>>        > Jan.Filipiak@trivago.com <mailto:
> Jan.Filipiak@trivago.com
> >>>>>
> >>>>>>>>>>
> >>>>>>>>>>        > >>>>> wrote:
> >>>>>>>>>>        > >>>>>
> >>>>>>>>>>        > >>>>> Thanks Adam for bringing Matthias to speed!
> >>>>>>>>>>        > >>>>>> about the differences. I think re-keying back
> >>> should be
> >>>>>>>>>>        optional at
> >>>>>>>>>>        > >>>>>> best.
> >>>>>>>>>>        > >>>>>> I would say we return a KScatteredTable with
> >>> reshuffle()
> >>>>>>>>>>        returning
> >>>>>>>>>>        > >>>>>> KTable<originalKey,Joined> to make the backwards
> >>>>>>>>>>        repartitioning
> >>>>>>>>>>        > >>>>>> optional.
> >>>>>>>>>>        > >>>>>> I am also in a big favour of doing the out of
> order
> >>>>>>>>>>        processing using
> >>>>>>>>>>        > >>>>>> group
> >>>>>>>>>>        > >>>>>> by instead high water mark tracking.
> >>>>>>>>>>        > >>>>>> Just because unbounded growth is just scary + It
> >>> saves
> >>>>>>>>>> us
> >>>>>>>>>>        the header
> >>>>>>>>>>        > >>>>>> stuff.
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>> I think the abstraction of always repartitioning
> >>> back is
> >>>>>>>>>>        just not so
> >>>>>>>>>>        > >>>>>> strong. Like the work has been done before we
> >>> partition
> >>>>>>>>>>        back and
> >>>>>>>>>>        > >>>>>> grouping
> >>>>>>>>>>        > >>>>>> by something else afterwards is really common.
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>>>>>>>>        > >>>>>>
> >>>>>>>>>>        > >>>>>> Hi Matthias
> >>>>>>>>>>        > >>>>>>> Thank you for your feedback, I do appreciate
> it!
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> While name spacing would be possible, it would
> >>> require
> >>>>>>>>>> to
> >>>>>>>>>>        > deserialize
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime overhead.
> I
> >>> would
> >>>>>>>>>>        suggest to
> >>>>>>>>>>        > no
> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
> this
> >>>>>>>>>>
> >>>>>>>>> becomes a
> >>>>>>>
> >>>>>>>>        > problem in
> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> later
> >>> on.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Agreed. I will go with using a reserved string
> >>> and
> >>>>>>>>>>        document it.
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> My main concern about the design it the type of
> >>> the
> >>>>>>>>>>        result KTable:
> >>>>>>>>>>        > If
> >>>>>>>>>>        > >>>>>>> I
> >>>>>>>>>>        > >>>>>>> understood the proposal correctly,
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> In your example, you have table1 and table2
> >>> swapped.
> >>>>>>>>>>        Here is how it
> >>>>>>>>>>        > >>>>>>> works
> >>>>>>>>>>        > >>>>>>> currently:
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> 1) table1 has the records that contain the
> >>> foreign key
> >>>>>>>>>>        within their
> >>>>>>>>>>        > >>>>>>> value.
> >>>>>>>>>>        > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >>>>>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>>>>        > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>>>>        > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> 2) A Value mapper is required to extract the
> >>> foreign
> >>>>>>>>>> key.
> >>>>>>>>>>        > >>>>>>> table1 foreign key mapper: ( value => value.fk
> >>>>>>>>>>        <http://value.fk> )
> >>>>>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> The mapper is applied to each element in
> table1,
> >>> and a
> >>>>>>>>>>        new combined
> >>>>>>>>>>        > >>>>>>> key is
> >>>>>>>>>>        > >>>>>>> made:
> >>>>>>>>>>        > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> >>>>>>>>>> (fk=A,bar=2)>,
> >>>>>>>>>>        <B-c,
> >>>>>>>>>>        > >>>>>>> (fk=B,bar=3)>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> 3) The rekeyed events are copartitioned with
> >>> table2:
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
> >>>>>>>>>>        (fk=A,bar=2)>
> >>>>>>>>>>        > >>>>>>> Table2: <A,X>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>>>>>        > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>>>>>>>>        > >>>>>>> Table2: <B,Y>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> 4) From here, they can be joined together
> locally
> >>> by
> >>>>>>>>>>        applying the
> >>>>>>>>>>        > >>>>>>> joiner
> >>>>>>>>>>        > >>>>>>> function.
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> At this point, Jan's design and my design
> >>> deviate. My
> >>>>>>>>>>        design goes
> >>>>>>>>>>        > on
> >>>>>>>>>>        > >>>>>>> to
> >>>>>>>>>>        > >>>>>>> repartition the data post-join and resolve
> >>> out-of-order
> >>>>>>>>>>        arrival of
> >>>>>>>>>>        > >>>>>>> records,
> >>>>>>>>>>        > >>>>>>> finally returning the data keyed just the
> >>> original key.
> >>>>>>>>>>        I do not
> >>>>>>>>>>        > >>>>>>> expose
> >>>>>>>>>>        > >>>>>>> the
> >>>>>>>>>>        > >>>>>>> CombinedKey or any of the internals outside of
> the
> >>>>>>>>>>        joinOnForeignKey
> >>>>>>>>>>        > >>>>>>> function. This does make for larger footprint,
> >>> but it
> >>>>>>>>>>        removes all
> >>>>>>>>>>        > >>>>>>> agency
> >>>>>>>>>>        > >>>>>>> for resolving out-of-order arrivals and
> handling
> >>>>>>>>>>        CombinedKeys from
> >>>>>>>>>>        > the
> >>>>>>>>>>        > >>>>>>> user. I believe that this makes the function
> much
> >>>>>>>>>> easier
> >>>>>>>>>>        to use.
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> Let me know if this helps resolve your
> questions,
> >>> and
> >>>>>>>>>>        please feel
> >>>>>>>>>>        > >>>>>>> free to
> >>>>>>>>>>        > >>>>>>> add anything else on your mind.
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> Thanks again,
> >>>>>>>>>>        > >>>>>>> Adam
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J.
> Sax <
> >>>>>>>>>>        > >>>>>>> matthias@confluent.io <mailto:
> >>> matthias@confluent.io>>
> >>>>>>>>>>
> >>>>>>>>>>        > >>>>>>> wrote:
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>> Hi,
> >>>>>>>>>>        > >>>>>>>
> >>>>>>>>>>        > >>>>>>>> I am just catching up on this thread. I did
> not
> >>> read
> >>>>>>>>>>        everything so
> >>>>>>>>>>        > >>>>>>>> far,
> >>>>>>>>>>        > >>>>>>>> but want to share couple of initial thoughts:
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Headers: I think there is a fundamental
> >>> difference
> >>>>>>>>>>        between header
> >>>>>>>>>>        > >>>>>>>> usage
> >>>>>>>>>>        > >>>>>>>> in this KIP and KP-258. For 258, we add
> headers
> >>> to
> >>>>>>>>>>        changelog topic
> >>>>>>>>>>        > >>>>>>>> that
> >>>>>>>>>>        > >>>>>>>> are owned by Kafka Streams and nobody else is
> >>> supposed
> >>>>>>>>>>        to write
> >>>>>>>>>>        > into
> >>>>>>>>>>        > >>>>>>>> them. In fact, no user header are written into
> >>> the
> >>>>>>>>>>        changelog topic
> >>>>>>>>>>        > >>>>>>>> and
> >>>>>>>>>>        > >>>>>>>> thus, there are not conflicts.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Nevertheless, I don't see a big issue with
> using
> >>>>>>>>>>        headers within
> >>>>>>>>>>        > >>>>>>>> Streams.
> >>>>>>>>>>        > >>>>>>>> As long as we document it, we can have some
> >>> "reserved"
> >>>>>>>>>>        header keys
> >>>>>>>>>>        > >>>>>>>> and
> >>>>>>>>>>        > >>>>>>>> users are not allowed to use when processing
> >>> data with
> >>>>>>>>>>        Kafka
> >>>>>>>>>>        > Streams.
> >>>>>>>>>>        > >>>>>>>> IMHO, this should be ok.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> I think there is a safe way to avoid
> conflicts,
> >>> since
> >>>>>>>>>> these
> >>>>>>>>>>        > headers
> >>>>>>>>>>        > >>>>>>>> are
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>> only needed in internal topics (I think):
> >>>>>>>>>>        > >>>>>>>>> For internal and changelog topics, we can
> >>> namespace
> >>>>>>>>>>        all headers:
> >>>>>>>>>>        > >>>>>>>>> * user-defined headers are namespaced as
> >>> "external."
> >>>>>>>>>> +
> >>>>>>>>>>        headerKey
> >>>>>>>>>>        > >>>>>>>>> * internal headers are namespaced as
> >>> "internal." +
> >>>>>>>>>>        headerKey
> >>>>>>>>>>        > >>>>>>>>>
> >>>>>>>>>>        > >>>>>>>>> While name spacing would be possible, it
> would
> >>>>>>>>>> require
> >>>>>>>>>>
> >>>>>>>>> to
> >>>>>>>
> >>>>>>>>        > >>>>>>>> deserialize
> >>>>>>>>>>        > >>>>>>>> user headers what implies a runtime overhead.
> I
> >>> would
> >>>>>>>>>>        suggest to
> >>>>>>>>>>        > no
> >>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If
> this
> >>>>>>>>>>
> >>>>>>>>> becomes a
> >>>>>>>
> >>>>>>>>        > problem in
> >>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing
> later
> >>> on.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> My main concern about the design it the type
> of
> >>> the
> >>>>>>>>>>        result KTable:
> >>>>>>>>>>        > >>>>>>>> If I
> >>>>>>>>>>        > >>>>>>>> understood the proposal correctly,
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>>>>        > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> KTable<K1,V3> joinedTable =
> >>> table1.join(table2,...);
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> implies that the `joinedTable` has the same
> key
> >>> as the
> >>>>>>>>>>        left input
> >>>>>>>>>>        > >>>>>>>> table.
> >>>>>>>>>>        > >>>>>>>> IMHO, this does not work because if table2
> >>> contains
> >>>>>>>>>>        multiple rows
> >>>>>>>>>>        > >>>>>>>> that
> >>>>>>>>>>        > >>>>>>>> join with a record in table1 (what is the main
> >>> purpose
> >>>>>>>>>>
> >>>>>>>>> of
> >>>>>>>
> >>>>>>>> a
> >>>>>>>>>>        > foreign
> >>>>>>>>>>        > >>>>>>>> key
> >>>>>>>>>>        > >>>>>>>> join), the result table would only contain a
> >>> single
> >>>>>>>>>>        join result,
> >>>>>>>>>>        > but
> >>>>>>>>>>        > >>>>>>>> not
> >>>>>>>>>>        > >>>>>>>> multiple.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Example:
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> table1 input stream: <A,X>
> >>>>>>>>>>        > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> We use table2 value a foreign key to table1
> key
> >>> (ie,
> >>>>>>>>>>        "A" joins).
> >>>>>>>>>>        > If
> >>>>>>>>>>        > >>>>>>>> the
> >>>>>>>>>>        > >>>>>>>> result key is the same key as key of table1,
> this
> >>>>>>>>>>        implies that the
> >>>>>>>>>>        > >>>>>>>> result can either be <A, join(X,1)> or <A,
> >>> join(X,2)>
> >>>>>>>>>>        but not
> >>>>>>>>>>        > both.
> >>>>>>>>>>        > >>>>>>>> Because the share the same key, whatever
> result
> >>> record
> >>>>>>>>>>        we emit
> >>>>>>>>>>        > later,
> >>>>>>>>>>        > >>>>>>>> overwrite the previous result.
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> This is the reason why Jan originally proposed
> >>> to use
> >>>>>>>>>> a
> >>>>>>>>>>        > combination
> >>>>>>>>>>        > >>>>>>>> of
> >>>>>>>>>>        > >>>>>>>> both primary keys of the input tables as key
> of
> >>> the
> >>>>>>>>>>        output table.
> >>>>>>>>>>        > >>>>>>>> This
> >>>>>>>>>>        > >>>>>>>> makes the keys of the output table unique and
> we
> >>> can
> >>>>>>>>>>        store both in
> >>>>>>>>>>        > >>>>>>>> the
> >>>>>>>>>>        > >>>>>>>> output table:
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
> >>> join(X,2)>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Thoughts?
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> -Matthias
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>>>>>>        > >>>>>>>>
> >>>>>>>>>>        > >>>>>>>> Just on remark here.
> >>>>>>>>>>        > >>>>>>>>> The high-watermark could be disregarded. The
> >>> decision
> >>>>>>>>>>        about the
> >>>>>>>>>>        > >>>>>>>>> forward
> >>>>>>>>>>        > >>>>>>>>> depends on the size of the aggregated map.
> >>>>>>>>>>        > >>>>>>>>> Only 1 element long maps would be unpacked
> and
> >>>>>>>>>>        forwarded. 0
> >>>>>>>>>>        > element
> >>>>>>>>>>        > >>>>>>>>> maps
> >>>>>>>>>>        > >>>>>>>>> would be published as delete. Any other count
> >>>>>>>>>>        > >>>>>>>>> of map entries is in "waiting for correct
> >>> deletes to
> >>>>>>>>>>        > arrive"-state.
> >>>>>>>>>>        > >>>>>>>>>
> >>>>>>>>>>        > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>>>>>        > >>>>>>>>>
> >>>>>>>>>>        > >>>>>>>>> It does look like I could replace the second
> >>>>>>>>>>        repartition store
> >>>>>>>>>>        > and
> >>>>>>>>>>        > >>>>>>>>>> highwater store with a groupBy and reduce.
> >>> However,
> >>>>>>>>>>        it looks
> >>>>>>>>>>        > like
> >>>>>>>>>>        > >>>>>>>>>> I
> >>>>>>>>>>        > >>>>>>>>>> would
> >>>>>>>>>>        > >>>>>>>>>> still need to store the highwater value
> within
> >>> the
> >>>>>>>>>>        materialized
> >>>>>>>>>>        > >>>>>>>>>> store,
> >>>>>>>>>>        > >>>>>>>>>>
> >>>>>>>>>>        > >>>>>>>>>> to
> >>>>>>>>>>        > >>>>>>>>> compare the arrival of out-of-order records
> >>> (assuming
> >>>>>>>>>>
> >>>>>>>>> my
> >>>>>>>
> >>>>>>>>        > >>>>>>>>> understanding
> >>>>>>>>>>        > >>>>>>>>> of
> >>>>>>>>>>        > >>>>>>>>> THIS is correct...). This in effect is the
> same
> >>> as
> >>>>>>>>>> the
> >>>>>>>>>>        design I
> >>>>>>>>>>        > have
> >>>>>>>>>>        > >>>>>>>>> now,
> >>>>>>>>>>        > >>>>>>>>> just with the two tables merged together.
> >>>>>>>>>>        > >>>>>>>>>
> >>>>>>>>>>        >
> >>>>>>>>>>        >
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>> --
> >>>>>>>> -- Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> --
> >>>>>>> -- Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Id say you can just call the vote.

that happens all the time, and if something comes up, it just goes back 
to discuss.

would not expect to much attention with another another email in this 
thread.

best Jan

On 09.10.2018 13:56, Adam Bellemare wrote:
> Hello Contributors
>
> I know that 2.1 is about to be released, but I do need to bump this to keep
> visibility up. I am still intending to push this through once contributor
> feedback is given.
>
> Main points that need addressing:
> 1) Any way (or benefit) in structuring the current singular graph node into
> multiple nodes? It has a whopping 25 parameters right now. I am a bit fuzzy
> on how the optimizations are supposed to work, so I would appreciate any
> help on this aspect.
>
> 2) Overall strategy for joining + resolving. This thread has much discourse
> between Jan and I between the current highwater mark proposal and a groupBy
> + reduce proposal. I am of the opinion that we need to strictly handle any
> chance of out-of-order data and leave none of it up to the consumer. Any
> comments or suggestions here would also help.
>
> 3) Anything else that you see that would prevent this from moving to a vote?
>
> Thanks
>
> Adam
>
>
>
>
>
>
>
> On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi Jan
>>
>> With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you
>> actually only need to specify the amount of segments you want and how large
>> they are. To the best of my understanding, what happens is that the
>> segments are automatically rolled over as new data with new timestamps are
>> created. We use this exact functionality in some of the work done
>> internally at my company. For reference, this is the hopping windowed store.
>>
>> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>>
>> In the code that I have provided, there are going to be two 24h segments.
>> When a record is put into the windowStore, it will be inserted at time T in
>> both segments. The two segments will always overlap by 12h. As time goes on
>> and new records are added (say at time T+12h+), the oldest segment will be
>> automatically deleted and a new segment created. The records are by default
>> inserted with the context.timestamp(), such that it is the record time, not
>> the clock time, which is used.
>>
>> To the best of my understanding, the timestamps are retained when
>> restoring from the changelog.
>>
>> Basically, this is heavy-handed way to deal with TTL at a segment-level,
>> instead of at an individual record level.
>>
>> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>>> Will that work? I expected it to blow up with ClassCastException or
>>> similar.
>>>
>>> You either would have to specify the window you fetch/put or iterate
>>> across all windows the key was found in right?
>>>
>>> I just hope the window-store doesn't check stream-time under the hoods
>>> that would be a questionable interface.
>>>
>>> If it does: did you see my comment on checking all the windows earlier?
>>> that would be needed to actually give reasonable time gurantees.
>>>
>>> Best
>>>
>>>
>>>
>>> On 25.09.2018 13:18, Adam Bellemare wrote:
>>>> Hi Jan
>>>>
>>>> Check for  " highwaterMat " in the PR. I only changed the state store,
>>> not
>>>> the ProcessorSupplier.
>>>>
>>>> Thanks,
>>>> Adam
>>>>
>>>> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>>>>
>>>> wrote:
>>>>
>>>>>
>>>>>
>>>>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>>>>
>>>>>> @Guozhang
>>>>>>
>>>>>> Thanks for the information. This is indeed something that will be
>>>>>> extremely
>>>>>> useful for this KIP.
>>>>>>
>>>>>> @Jan
>>>>>> Thanks for your explanations. That being said, I will not be moving
>>> ahead
>>>>>> with an implementation using reshuffle/groupBy solution as you
>>> propose.
>>>>>> That being said, if you wish to implement it yourself off of my
>>> current PR
>>>>>> and submit it as a competitive alternative, I would be more than
>>> happy to
>>>>>> help vet that as an alternate solution. As it stands right now, I do
>>> not
>>>>>> really have more time to invest into alternatives without there being
>>> a
>>>>>> strong indication from the binding voters which they would prefer.
>>>>>>
>>>>>>
>>>>> Hey, total no worries. I think I personally gave up on the streams DSL
>>> for
>>>>> some time already, otherwise I would have pulled this KIP through
>>> already.
>>>>> I am currently reimplementing my own DSL based on PAPI.
>>>>>
>>>>>
>>>>>> I will look at finishing up my PR with the windowed state store in the
>>>>>> next
>>>>>> week or so, exercising it via tests, and then I will come back for
>>> final
>>>>>> discussions. In the meantime, I hope that any of the binding voters
>>> could
>>>>>> take a look at the KIP in the wiki. I have updated it according to the
>>>>>> latest plan:
>>>>>>
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>>>>> Support+non-key+joining+in+KTable
>>>>>>
>>>>>> I have also updated the KIP PR to use a windowed store. This could be
>>>>>> replaced by the results of KIP-258 whenever they are completed.
>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>
>>>>> Is the HighWatermarkResolverProccessorsupplier already updated in the
>>> PR?
>>>>> expected it to change to Windowed<K>,Long Missing something?
>>>>>
>>>>>
>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>> Correction on my previous email: KAFKA-5533 is the wrong link, as it
>>> is
>>>>>>> for
>>>>>>> corresponding changelog mechanisms. But as part of KIP-258 we do
>>> want to
>>>>>>> have "handling out-of-order data for source KTable" such that
>>> instead of
>>>>>>> blindly apply the updates to the materialized store, i.e. following
>>>>>>> offset
>>>>>>> ordering, we will reject updates that are older than the current
>>> key's
>>>>>>> timestamps, i.e. following timestamp ordering.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hello Adam,
>>>>>>>>
>>>>>>>> Thanks for the explanation. Regarding the final step (i.e. the high
>>>>>>>> watermark store, now altered to be replaced with a window store), I
>>>>>>>> think
>>>>>>>> another current on-going KIP may actually help:
>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>>>>
>>>>>>>>
>>>>>>>> This is for adding the timestamp into a key-value store (i.e. only
>>> for
>>>>>>>> non-windowed KTable), and then one of its usage, as described in
>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
>>> then
>>>>>>>> "reject" updates from the source topics if its timestamp is smaller
>>> than
>>>>>>>> the current key's latest update timestamp. I think it is very
>>> similar to
>>>>>>>> what you have in mind for high watermark based filtering, while you
>>> only
>>>>>>>> need to make sure that the timestamps of the joining records are
>>>>>>>>
>>>>>>> correctly
>>>>>>>
>>>>>>>> inherited though the whole topology to the final stage.
>>>>>>>>
>>>>>>>> Note that this KIP is for key-value store and hence non-windowed
>>> KTables
>>>>>>>> only, but for windowed KTables we do not really have a good support
>>> for
>>>>>>>> their joins anyways (
>>> https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>>>>> I
>>>>>>>> think we can just consider non-windowed KTable-KTable non-key joins
>>> for
>>>>>>>> now. In which case, KIP-258 should help.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>>> Jan.Filipiak@trivago.com
>>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>>>>
>>>>>>>>> Hi Guozhang
>>>>>>>>>>
>>>>>>>>>> Current highwater mark implementation would grow endlessly based
>>> on
>>>>>>>>>> primary key of original event. It is a pair of (<this table
>>> primary
>>>>>>>>>>
>>>>>>>>> key>,
>>>>>>>
>>>>>>>> <highest offset seen for that key>). This is used to differentiate
>>>>>>>>>>
>>>>>>>>> between
>>>>>>>
>>>>>>>> late arrivals and new updates. My newest proposal would be to
>>> replace
>>>>>>>>>>
>>>>>>>>> it
>>>>>>>
>>>>>>>> with a Windowed state store of Duration N. This would allow the same
>>>>>>>>>> behaviour, but cap the size based on time. This should allow for
>>> all
>>>>>>>>>> late-arriving events to be processed, and should be customizable
>>> by
>>>>>>>>>> the
>>>>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
>>>>>>>>>>
>>>>>>>>> window,
>>>>>>>
>>>>>>>> or perhaps 7 days...).
>>>>>>>>>>
>>>>>>>>>> Hi Adam, using time based retention can do the trick here. Even
>>> if I
>>>>>>>>> would still like to see the automatic repartitioning optional
>>> since I
>>>>>>>>>
>>>>>>>> would
>>>>>>>
>>>>>>>> just reshuffle again. With windowed store I am a little bit
>>> sceptical
>>>>>>>>>
>>>>>>>> about
>>>>>>>
>>>>>>>> how to determine the window. So esentially one could run into
>>> problems
>>>>>>>>>
>>>>>>>> when
>>>>>>>
>>>>>>>> the rapid change happens near a window border. I will check you
>>>>>>>>> implementation in detail, if its problematic, we could still check
>>>>>>>>> _all_
>>>>>>>>> windows on read with not to bad performance impact I guess. Will
>>> let
>>>>>>>>> you
>>>>>>>>> know if the implementation would be correct as is. I wouldn't not
>>> like
>>>>>>>>>
>>>>>>>> to
>>>>>>>
>>>>>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B).
>>> I
>>>>>>>>>
>>>>>>>> think
>>>>>>>
>>>>>>>> we can't expect that.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> @Jan
>>>>>>>>>> I believe I understand what you mean now - thanks for the
>>> diagram, it
>>>>>>>>>> did really help. You are correct that I do not have the original
>>>>>>>>>>
>>>>>>>>> primary
>>>>>>>
>>>>>>>> key available, and I can see that if it was available then you
>>> would be
>>>>>>>>>> able to add and remove events from the Map. That being said, I
>>>>>>>>>>
>>>>>>>>> encourage
>>>>>>>
>>>>>>>> you to finish your diagrams / charts just for clarity for everyone
>>>>>>>>>>
>>>>>>>>> else.
>>>>>>>
>>>>>>>>
>>>>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
>>> understand
>>>>>>>>>>
>>>>>>>>> the benefits for the rest. Sorry about the original primary key, We
>>>>>>>>> have
>>>>>>>>> join and Group by implemented our own in PAPI and basically not
>>> using
>>>>>>>>>
>>>>>>>> any
>>>>>>>
>>>>>>>> DSL (Just the abstraction). Completely missed that in original DSL
>>> its
>>>>>>>>>
>>>>>>>> not
>>>>>>>
>>>>>>>> there and just assumed it. total brain mess up on my end. Will
>>> finish
>>>>>>>>>
>>>>>>>> the
>>>>>>>
>>>>>>>> chart as soon as i get a quite evening this week.
>>>>>>>>>
>>>>>>>>> My follow up question for you is, won't the Map stay inside the
>>> State
>>>>>>>>>
>>>>>>>>>> Store indefinitely after all of the changes have propagated? Isn't
>>>>>>>>>> this
>>>>>>>>>> effectively the same as a highwater mark state store?
>>>>>>>>>>
>>>>>>>>>> Thing is that if the map is empty, substractor is gonna return
>>> `null`
>>>>>>>>>
>>>>>>>> and
>>>>>>>
>>>>>>>> the key is removed from the keyspace. But there is going to be a
>>> store
>>>>>>>>> 100%, the good thing is that I can use this store directly for
>>>>>>>>> materialize() / enableSendingOldValues() is a regular store,
>>> satisfying
>>>>>>>>> all gurantees needed for further groupby / join. The Windowed
>>> store is
>>>>>>>>>
>>>>>>>> not
>>>>>>>
>>>>>>>> keeping the values, so for the next statefull operation we would
>>>>>>>>> need to instantiate an extra store. or we have the window store
>>> also
>>>>>>>>>
>>>>>>>> have
>>>>>>>
>>>>>>>> the values then.
>>>>>>>>>
>>>>>>>>> Long story short. if we can flip in a custom group by before
>>>>>>>>> repartitioning to the original primary key i think it would help
>>> the
>>>>>>>>>
>>>>>>>> users
>>>>>>>
>>>>>>>> big time in building efficient apps. Given the original primary key
>>>>>>>>>
>>>>>>>> issue I
>>>>>>>
>>>>>>>> understand that we do not have a solid foundation to build on.
>>>>>>>>> Leaving primary key carry along to the user. very unfortunate. I
>>> could
>>>>>>>>> understand the decision goes like that. I do not think its a good
>>>>>>>>>
>>>>>>>> decision.
>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>> Adam
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
>>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>>>        please remove me from this group
>>>>>>>>>>
>>>>>>>>>>        On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>>>>>
>>>>>>>>>>
>>>>>>>>>>        wrote:
>>>>>>>>>>
>>>>>>>>>>        > Hi Adam,
>>>>>>>>>>        >
>>>>>>>>>>        > give me some time, will make such a chart. last time i
>>> didn't
>>>>>>>>>>        get along
>>>>>>>>>>        > well with giphy and ruined all your charts.
>>>>>>>>>>        > Hopefully i can get it done today
>>>>>>>>>>        >
>>>>>>>>>>        > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>>>>>        > > Hi Jan
>>>>>>>>>>        > >
>>>>>>>>>>        > > I have included a diagram of what I attempted on the
>>> KIP.
>>>>>>>>>>        > >
>>>>>>>>>>        >
>>>>>>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>>>>        <
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>>>>        > >
>>>>>>>>>>        > > I attempted this back at the start of my own
>>> implementation
>>>>>>>>>> of
>>>>>>>>>>        this
>>>>>>>>>>        > > solution, and since I could not get it to work I have
>>> since
>>>>>>>>>>        discarded the
>>>>>>>>>>        > > code. At this point in time, if you wish to continue
>>> pursuing
>>>>>>>>>>        for your
>>>>>>>>>>        > > groupBy solution, I ask that you please create a
>>> diagram on
>>>>>>>>>>        the KIP
>>>>>>>>>>        > > carefully explaining your solution. Please feel free to
>>> use
>>>>>>>>>>        the image I
>>>>>>>>>>        > > just posted as a starting point. I am having trouble
>>>>>>>>>>        understanding your
>>>>>>>>>>        > > explanations but I think that a carefully constructed
>>> diagram
>>>>>>>>>>        will clear
>>>>>>>>>>        > up
>>>>>>>>>>        > > any misunderstandings. Alternately, please post a
>>>>>>>>>>        comprehensive PR with
>>>>>>>>>>        > > your solution. I can only guess at what you mean, and
>>> since I
>>>>>>>>>>        value my
>>>>>>>>>>        > own
>>>>>>>>>>        > > time as much as you value yours, I believe it is your
>>>>>>>>>>        responsibility to
>>>>>>>>>>        > > provide an implementation instead of me trying to guess.
>>>>>>>>>>        > >
>>>>>>>>>>        > > Adam
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > >
>>>>>>>>>>        > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>>>>>
>>>>>>>>>>
>>>>>>>>>>        > > wrote:
>>>>>>>>>>        > >
>>>>>>>>>>        > >> Hi James,
>>>>>>>>>>        > >>
>>>>>>>>>>        > >> nice to see you beeing interested. kafka streams at
>>> this
>>>>>>>>>>        point supports
>>>>>>>>>>        > >> all sorts of joins as long as both streams have the
>>> same
>>>>>>>>>> key.
>>>>>>>>>>        > >> Adam is currently implementing a join where a KTable
>>> and a
>>>>>>>>>>        KTable can
>>>>>>>>>>        > have
>>>>>>>>>>        > >> a one to many relation ship (1:n). We exploit that
>>> rocksdb
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>> a
>>>>>>>
>>>>>>>>        > >> datastore that keeps data sorted (At least exposes an
>>> API to
>>>>>>>>>>        access the
>>>>>>>>>>        > >> stored data in a sorted fashion).
>>>>>>>>>>        > >>
>>>>>>>>>>        > >> I think the technical caveats are well understood now
>>> and we
>>>>>>>>>>
>>>>>>>>> are
>>>>>>>
>>>>>>>>        > basically
>>>>>>>>>>        > >> down to philosophy and API Design ( when Adam sees my
>>> newest
>>>>>>>>>>        message).
>>>>>>>>>>        > >> I have a lengthy track record of loosing those kinda
>>>>>>>>>>        arguments within
>>>>>>>>>>        > the
>>>>>>>>>>        > >> streams community and I have no clue why. So I
>>> literally
>>>>>>>>>>        can't wait for
>>>>>>>>>>        > you
>>>>>>>>>>        > >> to churn through this thread and give you opinion on
>>> how we
>>>>>>>>>>        should
>>>>>>>>>>        > design
>>>>>>>>>>        > >> the return type of the oneToManyJoin and how many
>>> power we
>>>>>>>>>>        want to give
>>>>>>>>>>        > to
>>>>>>>>>>        > >> the user vs "simplicity" (where simplicity isn't
>>> really that
>>>>>>>>>>        as users
>>>>>>>>>>        > still
>>>>>>>>>>        > >> need to understand it I argue)
>>>>>>>>>>        > >>
>>>>>>>>>>        > >> waiting for you to join in on the discussion
>>>>>>>>>>        > >>
>>>>>>>>>>        > >> Best Jan
>>>>>>>>>>        > >>
>>>>>>>>>>        > >>
>>>>>>>>>>        > >>
>>>>>>>>>>        > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>>>>>        > >>
>>>>>>>>>>        > >>> I am new to this group and I found this subject
>>>>>>>>>>        interesting.  Sounds
>>>>>>>>>>        > like
>>>>>>>>>>        > >>> you guys want to implement a join table of two
>>> streams? Is
>>>>>>>>>> there
>>>>>>>>>>        > somewhere
>>>>>>>>>>        > >>> I can see the original requirement or proposal?
>>>>>>>>>>        > >>>
>>>>>>>>>>        > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>>>>>        <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>>>>>
>>>>>>>>>>
>>>>>>>>>>        > >>>> wrote:
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>> I'm currently testing using a Windowed Store to
>>> store the
>>>>>>>>>>        highwater
>>>>>>>>>>        > >>>>> mark.
>>>>>>>>>>        > >>>>> By all indications this should work fine, with the
>>> caveat
>>>>>>>>>>        being that
>>>>>>>>>>        > it
>>>>>>>>>>        > >>>>> can
>>>>>>>>>>        > >>>>> only resolve out-of-order arrival for up to the
>>> size of
>>>>>>>>>>        the window
>>>>>>>>>>        > (ie:
>>>>>>>>>>        > >>>>> 24h, 72h, etc). This would remove the possibility
>>> of it
>>>>>>>>>>
>>>>>>>>> being
>>>>>>>
>>>>>>>>        > unbounded
>>>>>>>>>>        > >>>>> in
>>>>>>>>>>        > >>>>> size.
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>> With regards to Jan's suggestion, I believe this is
>>> where
>>>>>>>>>>        we will
>>>>>>>>>>        > have
>>>>>>>>>>        > >>>>> to
>>>>>>>>>>        > >>>>> remain in disagreement. While I do not disagree
>>> with your
>>>>>>>>>>        statement
>>>>>>>>>>        > >>>>> about
>>>>>>>>>>        > >>>>> there likely to be additional joins done in a
>>> real-world
>>>>>>>>>>        workflow, I
>>>>>>>>>>        > do
>>>>>>>>>>        > >>>>> not
>>>>>>>>>>        > >>>>> see how you can conclusively deal with out-of-order
>>>>>>>>>> arrival
>>>>>>>>>> of
>>>>>>>>>>        > >>>>> foreign-key
>>>>>>>>>>        > >>>>> changes and subsequent joins. I have attempted what
>>> I
>>>>>>>>>>        think you have
>>>>>>>>>>        > >>>>> proposed (without a high-water, using groupBy and
>>> reduce)
>>>>>>>>>>        and found
>>>>>>>>>>        > >>>>> that if
>>>>>>>>>>        > >>>>> the foreign key changes too quickly, or the load on
>>> a
>>>>>>>>>>        stream thread
>>>>>>>>>>        > is
>>>>>>>>>>        > >>>>> too
>>>>>>>>>>        > >>>>> high, the joined messages will arrive out-of-order
>>> and be
>>>>>>>>>>        incorrectly
>>>>>>>>>>        > >>>>> propagated, such that an intermediate event is
>>>>>>>>>> represented
>>>>>>>>>>        as the
>>>>>>>>>>        > final
>>>>>>>>>>        > >>>>> event.
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>> Can you shed some light on your groupBy
>>> implementation.
>>>>>>>>>>        There must be
>>>>>>>>>>        > >>>> some sort of flaw in it.
>>>>>>>>>>        > >>>> I have a suspicion where it is, I would just like to
>>>>>>>>>>        confirm. The idea
>>>>>>>>>>        > >>>> is bullet proof and it must be
>>>>>>>>>>        > >>>> an implementation mess up. I would like to clarify
>>> before
>>>>>>>>>>        we draw a
>>>>>>>>>>        > >>>> conclusion.
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>    Repartitioning the scattered events back to their
>>>>>>>>>>
>>>>>>>>> original
>>>>>>>
>>>>>>>>        > >>>>> partitions is the only way I know how to conclusively
>>> deal
>>>>>>>>>>        with
>>>>>>>>>>        > >>>>> out-of-order events in a given time frame, and to
>>> ensure
>>>>>>>>>>        that the
>>>>>>>>>>        > data
>>>>>>>>>>        > >>>>> is
>>>>>>>>>>        > >>>>> eventually consistent with the input events.
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>> If you have some code to share that illustrates your
>>>>>>>>>>        approach, I
>>>>>>>>>>        > would
>>>>>>>>>>        > >>>>> be
>>>>>>>>>>        > >>>>> very grateful as it would remove any
>>> misunderstandings
>>>>>>>>>>        that I may
>>>>>>>>>>        > have.
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>> ah okay you were looking for my code. I don't have
>>>>>>>>>>        something easily
>>>>>>>>>>        > >>>> readable here as its bloated with OO-patterns.
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>> its anyhow trivial:
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>> @Override
>>>>>>>>>>        > >>>>      public T apply(K aggKey, V value, T aggregate)
>>>>>>>>>>        > >>>>      {
>>>>>>>>>>        > >>>>          Map<U, V> currentStateAsMap =
>>> asMap(aggregate);
>>>>>>>>>> <<
>>>>>>>>>>        imaginary
>>>>>>>>>>        > >>>>          U toModifyKey = mapper.apply(value);
>>>>>>>>>>        > >>>>              << this is the place where people
>>> actually
>>>>>>>>>>        gonna have
>>>>>>>>>>        > issues
>>>>>>>>>>        > >>>> and why you probably couldn't do it. we would need
>>> to find
>>>>>>>>>>        a solution
>>>>>>>>>>        > here.
>>>>>>>>>>        > >>>> I didn't realize that yet.
>>>>>>>>>>        > >>>>              << we propagate the field in the
>>> joiner, so
>>>>>>>>>>        that we can
>>>>>>>>>>        > pick
>>>>>>>>>>        > >>>> it up in an aggregate. Probably you have not thought
>>> of
>>>>>>>>>>        this in your
>>>>>>>>>>        > >>>> approach right?
>>>>>>>>>>        > >>>>              << I am very open to find a generic
>>> solution
>>>>>>>>>>        here. In my
>>>>>>>>>>        > >>>> honest opinion this is broken in KTableImpl.GroupBy
>>> that
>>>>>>>>>> it
>>>>>>>>>>        looses
>>>>>>>>>>        > the keys
>>>>>>>>>>        > >>>> and only maintains the aggregate key.
>>>>>>>>>>        > >>>>              << I abstracted it away back then way
>>> before
>>>>>>>>>> i
>>>>>>>>>> was
>>>>>>>>>>        > thinking
>>>>>>>>>>        > >>>> of oneToMany join. That is why I didn't realize its
>>>>>>>>>>        significance here.
>>>>>>>>>>        > >>>>              << Opinions?
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>          for (V m : current)
>>>>>>>>>>        > >>>>          {
>>>>>>>>>>        > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>>>>>>        > >>>>          }
>>>>>>>>>>        > >>>>          if (isAdder)
>>>>>>>>>>        > >>>>          {
>>>>>>>>>>        > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>>>>>>        > >>>>          }
>>>>>>>>>>        > >>>>          else
>>>>>>>>>>        > >>>>          {
>>>>>>>>>>        > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>>>>        > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>>>>>        > >>>>                  return null;
>>>>>>>>>>        > >>>>              }
>>>>>>>>>>        > >>>>          }
>>>>>>>>>>        > >>>>          retrun asAggregateType(currentStateAsMap)
>>>>>>>>>>        > >>>>      }
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>>
>>>>>>>>>>        > >>>> Thanks,
>>>>>>>>>>        > >>>>> Adam
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>>>>>>>>        > Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>>>>>
>>>>>>>>>>
>>>>>>>>>>        > >>>>> wrote:
>>>>>>>>>>        > >>>>>
>>>>>>>>>>        > >>>>> Thanks Adam for bringing Matthias to speed!
>>>>>>>>>>        > >>>>>> about the differences. I think re-keying back
>>> should be
>>>>>>>>>>        optional at
>>>>>>>>>>        > >>>>>> best.
>>>>>>>>>>        > >>>>>> I would say we return a KScatteredTable with
>>> reshuffle()
>>>>>>>>>>        returning
>>>>>>>>>>        > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>>>>>>>>        repartitioning
>>>>>>>>>>        > >>>>>> optional.
>>>>>>>>>>        > >>>>>> I am also in a big favour of doing the out of order
>>>>>>>>>>        processing using
>>>>>>>>>>        > >>>>>> group
>>>>>>>>>>        > >>>>>> by instead high water mark tracking.
>>>>>>>>>>        > >>>>>> Just because unbounded growth is just scary + It
>>> saves
>>>>>>>>>> us
>>>>>>>>>>        the header
>>>>>>>>>>        > >>>>>> stuff.
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>> I think the abstraction of always repartitioning
>>> back is
>>>>>>>>>>        just not so
>>>>>>>>>>        > >>>>>> strong. Like the work has been done before we
>>> partition
>>>>>>>>>>        back and
>>>>>>>>>>        > >>>>>> grouping
>>>>>>>>>>        > >>>>>> by something else afterwards is really common.
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>>>>>>        > >>>>>>
>>>>>>>>>>        > >>>>>> Hi Matthias
>>>>>>>>>>        > >>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> While name spacing would be possible, it would
>>> require
>>>>>>>>>> to
>>>>>>>>>>        > deserialize
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>> user headers what implies a runtime overhead. I
>>> would
>>>>>>>>>>        suggest to
>>>>>>>>>>        > no
>>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>>>>>
>>>>>>>>> becomes a
>>>>>>>
>>>>>>>>        > problem in
>>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing later
>>> on.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Agreed. I will go with using a reserved string
>>> and
>>>>>>>>>>        document it.
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> My main concern about the design it the type of
>>> the
>>>>>>>>>>        result KTable:
>>>>>>>>>>        > If
>>>>>>>>>>        > >>>>>>> I
>>>>>>>>>>        > >>>>>>> understood the proposal correctly,
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> In your example, you have table1 and table2
>>> swapped.
>>>>>>>>>>        Here is how it
>>>>>>>>>>        > >>>>>>> works
>>>>>>>>>>        > >>>>>>> currently:
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> 1) table1 has the records that contain the
>>> foreign key
>>>>>>>>>>        within their
>>>>>>>>>>        > >>>>>>> value.
>>>>>>>>>>        > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>>>>        > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>>>>        > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> 2) A Value mapper is required to extract the
>>> foreign
>>>>>>>>>> key.
>>>>>>>>>>        > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>>>>>>>>        <http://value.fk> )
>>>>>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> The mapper is applied to each element in table1,
>>> and a
>>>>>>>>>>        new combined
>>>>>>>>>>        > >>>>>>> key is
>>>>>>>>>>        > >>>>>>> made:
>>>>>>>>>>        > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>>>>> (fk=A,bar=2)>,
>>>>>>>>>>        <B-c,
>>>>>>>>>>        > >>>>>>> (fk=B,bar=3)>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> 3) The rekeyed events are copartitioned with
>>> table2:
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>>>>>        > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>>>>>        (fk=A,bar=2)>
>>>>>>>>>>        > >>>>>>> Table2: <A,X>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>>>>>        > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>>>>>        > >>>>>>> Table2: <B,Y>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> 4) From here, they can be joined together locally
>>> by
>>>>>>>>>>        applying the
>>>>>>>>>>        > >>>>>>> joiner
>>>>>>>>>>        > >>>>>>> function.
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> At this point, Jan's design and my design
>>> deviate. My
>>>>>>>>>>        design goes
>>>>>>>>>>        > on
>>>>>>>>>>        > >>>>>>> to
>>>>>>>>>>        > >>>>>>> repartition the data post-join and resolve
>>> out-of-order
>>>>>>>>>>        arrival of
>>>>>>>>>>        > >>>>>>> records,
>>>>>>>>>>        > >>>>>>> finally returning the data keyed just the
>>> original key.
>>>>>>>>>>        I do not
>>>>>>>>>>        > >>>>>>> expose
>>>>>>>>>>        > >>>>>>> the
>>>>>>>>>>        > >>>>>>> CombinedKey or any of the internals outside of the
>>>>>>>>>>        joinOnForeignKey
>>>>>>>>>>        > >>>>>>> function. This does make for larger footprint,
>>> but it
>>>>>>>>>>        removes all
>>>>>>>>>>        > >>>>>>> agency
>>>>>>>>>>        > >>>>>>> for resolving out-of-order arrivals and handling
>>>>>>>>>>        CombinedKeys from
>>>>>>>>>>        > the
>>>>>>>>>>        > >>>>>>> user. I believe that this makes the function much
>>>>>>>>>> easier
>>>>>>>>>>        to use.
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> Let me know if this helps resolve your questions,
>>> and
>>>>>>>>>>        please feel
>>>>>>>>>>        > >>>>>>> free to
>>>>>>>>>>        > >>>>>>> add anything else on your mind.
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> Thanks again,
>>>>>>>>>>        > >>>>>>> Adam
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>>>>>>        > >>>>>>> matthias@confluent.io <mailto:
>>> matthias@confluent.io>>
>>>>>>>>>>
>>>>>>>>>>        > >>>>>>> wrote:
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>> Hi,
>>>>>>>>>>        > >>>>>>>
>>>>>>>>>>        > >>>>>>>> I am just catching up on this thread. I did not
>>> read
>>>>>>>>>>        everything so
>>>>>>>>>>        > >>>>>>>> far,
>>>>>>>>>>        > >>>>>>>> but want to share couple of initial thoughts:
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Headers: I think there is a fundamental
>>> difference
>>>>>>>>>>        between header
>>>>>>>>>>        > >>>>>>>> usage
>>>>>>>>>>        > >>>>>>>> in this KIP and KP-258. For 258, we add headers
>>> to
>>>>>>>>>>        changelog topic
>>>>>>>>>>        > >>>>>>>> that
>>>>>>>>>>        > >>>>>>>> are owned by Kafka Streams and nobody else is
>>> supposed
>>>>>>>>>>        to write
>>>>>>>>>>        > into
>>>>>>>>>>        > >>>>>>>> them. In fact, no user header are written into
>>> the
>>>>>>>>>>        changelog topic
>>>>>>>>>>        > >>>>>>>> and
>>>>>>>>>>        > >>>>>>>> thus, there are not conflicts.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Nevertheless, I don't see a big issue with using
>>>>>>>>>>        headers within
>>>>>>>>>>        > >>>>>>>> Streams.
>>>>>>>>>>        > >>>>>>>> As long as we document it, we can have some
>>> "reserved"
>>>>>>>>>>        header keys
>>>>>>>>>>        > >>>>>>>> and
>>>>>>>>>>        > >>>>>>>> users are not allowed to use when processing
>>> data with
>>>>>>>>>>        Kafka
>>>>>>>>>>        > Streams.
>>>>>>>>>>        > >>>>>>>> IMHO, this should be ok.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> I think there is a safe way to avoid conflicts,
>>> since
>>>>>>>>>> these
>>>>>>>>>>        > headers
>>>>>>>>>>        > >>>>>>>> are
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>> only needed in internal topics (I think):
>>>>>>>>>>        > >>>>>>>>> For internal and changelog topics, we can
>>> namespace
>>>>>>>>>>        all headers:
>>>>>>>>>>        > >>>>>>>>> * user-defined headers are namespaced as
>>> "external."
>>>>>>>>>> +
>>>>>>>>>>        headerKey
>>>>>>>>>>        > >>>>>>>>> * internal headers are namespaced as
>>> "internal." +
>>>>>>>>>>        headerKey
>>>>>>>>>>        > >>>>>>>>>
>>>>>>>>>>        > >>>>>>>>> While name spacing would be possible, it would
>>>>>>>>>> require
>>>>>>>>>>
>>>>>>>>> to
>>>>>>>
>>>>>>>>        > >>>>>>>> deserialize
>>>>>>>>>>        > >>>>>>>> user headers what implies a runtime overhead. I
>>> would
>>>>>>>>>>        suggest to
>>>>>>>>>>        > no
>>>>>>>>>>        > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>>>>>
>>>>>>>>> becomes a
>>>>>>>
>>>>>>>>        > problem in
>>>>>>>>>>        > >>>>>>>> the future, we can still add name spacing later
>>> on.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> My main concern about the design it the type of
>>> the
>>>>>>>>>>        result KTable:
>>>>>>>>>>        > >>>>>>>> If I
>>>>>>>>>>        > >>>>>>>> understood the proposal correctly,
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>>>>        > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> KTable<K1,V3> joinedTable =
>>> table1.join(table2,...);
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> implies that the `joinedTable` has the same key
>>> as the
>>>>>>>>>>        left input
>>>>>>>>>>        > >>>>>>>> table.
>>>>>>>>>>        > >>>>>>>> IMHO, this does not work because if table2
>>> contains
>>>>>>>>>>        multiple rows
>>>>>>>>>>        > >>>>>>>> that
>>>>>>>>>>        > >>>>>>>> join with a record in table1 (what is the main
>>> purpose
>>>>>>>>>>
>>>>>>>>> of
>>>>>>>
>>>>>>>> a
>>>>>>>>>>        > foreign
>>>>>>>>>>        > >>>>>>>> key
>>>>>>>>>>        > >>>>>>>> join), the result table would only contain a
>>> single
>>>>>>>>>>        join result,
>>>>>>>>>>        > but
>>>>>>>>>>        > >>>>>>>> not
>>>>>>>>>>        > >>>>>>>> multiple.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Example:
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> table1 input stream: <A,X>
>>>>>>>>>>        > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> We use table2 value a foreign key to table1 key
>>> (ie,
>>>>>>>>>>        "A" joins).
>>>>>>>>>>        > If
>>>>>>>>>>        > >>>>>>>> the
>>>>>>>>>>        > >>>>>>>> result key is the same key as key of table1, this
>>>>>>>>>>        implies that the
>>>>>>>>>>        > >>>>>>>> result can either be <A, join(X,1)> or <A,
>>> join(X,2)>
>>>>>>>>>>        but not
>>>>>>>>>>        > both.
>>>>>>>>>>        > >>>>>>>> Because the share the same key, whatever result
>>> record
>>>>>>>>>>        we emit
>>>>>>>>>>        > later,
>>>>>>>>>>        > >>>>>>>> overwrite the previous result.
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> This is the reason why Jan originally proposed
>>> to use
>>>>>>>>>> a
>>>>>>>>>>        > combination
>>>>>>>>>>        > >>>>>>>> of
>>>>>>>>>>        > >>>>>>>> both primary keys of the input tables as key of
>>> the
>>>>>>>>>>        output table.
>>>>>>>>>>        > >>>>>>>> This
>>>>>>>>>>        > >>>>>>>> makes the keys of the output table unique and we
>>> can
>>>>>>>>>>        store both in
>>>>>>>>>>        > >>>>>>>> the
>>>>>>>>>>        > >>>>>>>> output table:
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>>> join(X,2)>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Thoughts?
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> -Matthias
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>>>>        > >>>>>>>>
>>>>>>>>>>        > >>>>>>>> Just on remark here.
>>>>>>>>>>        > >>>>>>>>> The high-watermark could be disregarded. The
>>> decision
>>>>>>>>>>        about the
>>>>>>>>>>        > >>>>>>>>> forward
>>>>>>>>>>        > >>>>>>>>> depends on the size of the aggregated map.
>>>>>>>>>>        > >>>>>>>>> Only 1 element long maps would be unpacked and
>>>>>>>>>>        forwarded. 0
>>>>>>>>>>        > element
>>>>>>>>>>        > >>>>>>>>> maps
>>>>>>>>>>        > >>>>>>>>> would be published as delete. Any other count
>>>>>>>>>>        > >>>>>>>>> of map entries is in "waiting for correct
>>> deletes to
>>>>>>>>>>        > arrive"-state.
>>>>>>>>>>        > >>>>>>>>>
>>>>>>>>>>        > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>>>>        > >>>>>>>>>
>>>>>>>>>>        > >>>>>>>>> It does look like I could replace the second
>>>>>>>>>>        repartition store
>>>>>>>>>>        > and
>>>>>>>>>>        > >>>>>>>>>> highwater store with a groupBy and reduce.
>>> However,
>>>>>>>>>>        it looks
>>>>>>>>>>        > like
>>>>>>>>>>        > >>>>>>>>>> I
>>>>>>>>>>        > >>>>>>>>>> would
>>>>>>>>>>        > >>>>>>>>>> still need to store the highwater value within
>>> the
>>>>>>>>>>        materialized
>>>>>>>>>>        > >>>>>>>>>> store,
>>>>>>>>>>        > >>>>>>>>>>
>>>>>>>>>>        > >>>>>>>>>> to
>>>>>>>>>>        > >>>>>>>>> compare the arrival of out-of-order records
>>> (assuming
>>>>>>>>>>
>>>>>>>>> my
>>>>>>>
>>>>>>>>        > >>>>>>>>> understanding
>>>>>>>>>>        > >>>>>>>>> of
>>>>>>>>>>        > >>>>>>>>> THIS is correct...). This in effect is the same
>>> as
>>>>>>>>>> the
>>>>>>>>>>        design I
>>>>>>>>>>        > have
>>>>>>>>>>        > >>>>>>>>> now,
>>>>>>>>>>        > >>>>>>>>> just with the two tables merged together.
>>>>>>>>>>        > >>>>>>>>>
>>>>>>>>>>        >
>>>>>>>>>>        >
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> -- Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> -- Guozhang
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hello Contributors

I know that 2.1 is about to be released, but I do need to bump this to keep
visibility up. I am still intending to push this through once contributor
feedback is given.

Main points that need addressing:
1) Any way (or benefit) in structuring the current singular graph node into
multiple nodes? It has a whopping 25 parameters right now. I am a bit fuzzy
on how the optimizations are supposed to work, so I would appreciate any
help on this aspect.

2) Overall strategy for joining + resolving. This thread has much discourse
between Jan and I between the current highwater mark proposal and a groupBy
+ reduce proposal. I am of the opinion that we need to strictly handle any
chance of out-of-order data and leave none of it up to the consumer. Any
comments or suggestions here would also help.

3) Anything else that you see that would prevent this from moving to a vote?

Thanks

Adam







On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Jan
>
> With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you
> actually only need to specify the amount of segments you want and how large
> they are. To the best of my understanding, what happens is that the
> segments are automatically rolled over as new data with new timestamps are
> created. We use this exact functionality in some of the work done
> internally at my company. For reference, this is the hopping windowed store.
>
> https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21
>
> In the code that I have provided, there are going to be two 24h segments.
> When a record is put into the windowStore, it will be inserted at time T in
> both segments. The two segments will always overlap by 12h. As time goes on
> and new records are added (say at time T+12h+), the oldest segment will be
> automatically deleted and a new segment created. The records are by default
> inserted with the context.timestamp(), such that it is the record time, not
> the clock time, which is used.
>
> To the best of my understanding, the timestamps are retained when
> restoring from the changelog.
>
> Basically, this is heavy-handed way to deal with TTL at a segment-level,
> instead of at an individual record level.
>
> On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Will that work? I expected it to blow up with ClassCastException or
>> similar.
>>
>> You either would have to specify the window you fetch/put or iterate
>> across all windows the key was found in right?
>>
>> I just hope the window-store doesn't check stream-time under the hoods
>> that would be a questionable interface.
>>
>> If it does: did you see my comment on checking all the windows earlier?
>> that would be needed to actually give reasonable time gurantees.
>>
>> Best
>>
>>
>>
>> On 25.09.2018 13:18, Adam Bellemare wrote:
>> > Hi Jan
>> >
>> > Check for  " highwaterMat " in the PR. I only changed the state store,
>> not
>> > the ProcessorSupplier.
>> >
>> > Thanks,
>> > Adam
>> >
>> > On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>> >
>> > wrote:
>> >
>> >>
>> >>
>> >> On 24.09.2018 16:26, Adam Bellemare wrote:
>> >>
>> >>> @Guozhang
>> >>>
>> >>> Thanks for the information. This is indeed something that will be
>> >>> extremely
>> >>> useful for this KIP.
>> >>>
>> >>> @Jan
>> >>> Thanks for your explanations. That being said, I will not be moving
>> ahead
>> >>> with an implementation using reshuffle/groupBy solution as you
>> propose.
>> >>> That being said, if you wish to implement it yourself off of my
>> current PR
>> >>> and submit it as a competitive alternative, I would be more than
>> happy to
>> >>> help vet that as an alternate solution. As it stands right now, I do
>> not
>> >>> really have more time to invest into alternatives without there being
>> a
>> >>> strong indication from the binding voters which they would prefer.
>> >>>
>> >>>
>> >> Hey, total no worries. I think I personally gave up on the streams DSL
>> for
>> >> some time already, otherwise I would have pulled this KIP through
>> already.
>> >> I am currently reimplementing my own DSL based on PAPI.
>> >>
>> >>
>> >>> I will look at finishing up my PR with the windowed state store in the
>> >>> next
>> >>> week or so, exercising it via tests, and then I will come back for
>> final
>> >>> discussions. In the meantime, I hope that any of the binding voters
>> could
>> >>> take a look at the KIP in the wiki. I have updated it according to the
>> >>> latest plan:
>> >>>
>> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> >>> Support+non-key+joining+in+KTable
>> >>>
>> >>> I have also updated the KIP PR to use a windowed store. This could be
>> >>> replaced by the results of KIP-258 whenever they are completed.
>> >>> https://github.com/apache/kafka/pull/5527
>> >>>
>> >>> Thanks,
>> >>>
>> >>> Adam
>> >>>
>> >>
>> >> Is the HighWatermarkResolverProccessorsupplier already updated in the
>> PR?
>> >> expected it to change to Windowed<K>,Long Missing something?
>> >>
>> >>
>> >>
>> >>>
>> >>>
>> >>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
>> >>> wrote:
>> >>>
>> >>> Correction on my previous email: KAFKA-5533 is the wrong link, as it
>> is
>> >>>> for
>> >>>> corresponding changelog mechanisms. But as part of KIP-258 we do
>> want to
>> >>>> have "handling out-of-order data for source KTable" such that
>> instead of
>> >>>> blindly apply the updates to the materialized store, i.e. following
>> >>>> offset
>> >>>> ordering, we will reject updates that are older than the current
>> key's
>> >>>> timestamps, i.e. following timestamp ordering.
>> >>>>
>> >>>>
>> >>>> Guozhang
>> >>>>
>> >>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
>> >>>> wrote:
>> >>>>
>> >>>> Hello Adam,
>> >>>>>
>> >>>>> Thanks for the explanation. Regarding the final step (i.e. the high
>> >>>>> watermark store, now altered to be replaced with a window store), I
>> >>>>> think
>> >>>>> another current on-going KIP may actually help:
>> >>>>>
>> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>> >>>>>
>> >>>>>
>> >>>>> This is for adding the timestamp into a key-value store (i.e. only
>> for
>> >>>>> non-windowed KTable), and then one of its usage, as described in
>> >>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
>> then
>> >>>>> "reject" updates from the source topics if its timestamp is smaller
>> than
>> >>>>> the current key's latest update timestamp. I think it is very
>> similar to
>> >>>>> what you have in mind for high watermark based filtering, while you
>> only
>> >>>>> need to make sure that the timestamps of the joining records are
>> >>>>>
>> >>>> correctly
>> >>>>
>> >>>>> inherited though the whole topology to the final stage.
>> >>>>>
>> >>>>> Note that this KIP is for key-value store and hence non-windowed
>> KTables
>> >>>>> only, but for windowed KTables we do not really have a good support
>> for
>> >>>>> their joins anyways (
>> https://issues.apache.org/jira/browse/KAFKA-7107)
>> >>>>> I
>> >>>>> think we can just consider non-windowed KTable-KTable non-key joins
>> for
>> >>>>> now. In which case, KIP-258 should help.
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> Guozhang
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
>> Jan.Filipiak@trivago.com
>> >>>>>>
>> >>>>> wrote:
>> >>>>>
>> >>>>>
>> >>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>> >>>>>>
>> >>>>>> Hi Guozhang
>> >>>>>>>
>> >>>>>>> Current highwater mark implementation would grow endlessly based
>> on
>> >>>>>>> primary key of original event. It is a pair of (<this table
>> primary
>> >>>>>>>
>> >>>>>> key>,
>> >>>>
>> >>>>> <highest offset seen for that key>). This is used to differentiate
>> >>>>>>>
>> >>>>>> between
>> >>>>
>> >>>>> late arrivals and new updates. My newest proposal would be to
>> replace
>> >>>>>>>
>> >>>>>> it
>> >>>>
>> >>>>> with a Windowed state store of Duration N. This would allow the same
>> >>>>>>> behaviour, but cap the size based on time. This should allow for
>> all
>> >>>>>>> late-arriving events to be processed, and should be customizable
>> by
>> >>>>>>> the
>> >>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
>> >>>>>>>
>> >>>>>> window,
>> >>>>
>> >>>>> or perhaps 7 days...).
>> >>>>>>>
>> >>>>>>> Hi Adam, using time based retention can do the trick here. Even
>> if I
>> >>>>>> would still like to see the automatic repartitioning optional
>> since I
>> >>>>>>
>> >>>>> would
>> >>>>
>> >>>>> just reshuffle again. With windowed store I am a little bit
>> sceptical
>> >>>>>>
>> >>>>> about
>> >>>>
>> >>>>> how to determine the window. So esentially one could run into
>> problems
>> >>>>>>
>> >>>>> when
>> >>>>
>> >>>>> the rapid change happens near a window border. I will check you
>> >>>>>> implementation in detail, if its problematic, we could still check
>> >>>>>> _all_
>> >>>>>> windows on read with not to bad performance impact I guess. Will
>> let
>> >>>>>> you
>> >>>>>> know if the implementation would be correct as is. I wouldn't not
>> like
>> >>>>>>
>> >>>>> to
>> >>>>
>> >>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B).
>> I
>> >>>>>>
>> >>>>> think
>> >>>>
>> >>>>> we can't expect that.
>> >>>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>> @Jan
>> >>>>>>> I believe I understand what you mean now - thanks for the
>> diagram, it
>> >>>>>>> did really help. You are correct that I do not have the original
>> >>>>>>>
>> >>>>>> primary
>> >>>>
>> >>>>> key available, and I can see that if it was available then you
>> would be
>> >>>>>>> able to add and remove events from the Map. That being said, I
>> >>>>>>>
>> >>>>>> encourage
>> >>>>
>> >>>>> you to finish your diagrams / charts just for clarity for everyone
>> >>>>>>>
>> >>>>>> else.
>> >>>>
>> >>>>>
>> >>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
>> understand
>> >>>>>>>
>> >>>>>> the benefits for the rest. Sorry about the original primary key, We
>> >>>>>> have
>> >>>>>> join and Group by implemented our own in PAPI and basically not
>> using
>> >>>>>>
>> >>>>> any
>> >>>>
>> >>>>> DSL (Just the abstraction). Completely missed that in original DSL
>> its
>> >>>>>>
>> >>>>> not
>> >>>>
>> >>>>> there and just assumed it. total brain mess up on my end. Will
>> finish
>> >>>>>>
>> >>>>> the
>> >>>>
>> >>>>> chart as soon as i get a quite evening this week.
>> >>>>>>
>> >>>>>> My follow up question for you is, won't the Map stay inside the
>> State
>> >>>>>>
>> >>>>>>> Store indefinitely after all of the changes have propagated? Isn't
>> >>>>>>> this
>> >>>>>>> effectively the same as a highwater mark state store?
>> >>>>>>>
>> >>>>>>> Thing is that if the map is empty, substractor is gonna return
>> `null`
>> >>>>>>
>> >>>>> and
>> >>>>
>> >>>>> the key is removed from the keyspace. But there is going to be a
>> store
>> >>>>>> 100%, the good thing is that I can use this store directly for
>> >>>>>> materialize() / enableSendingOldValues() is a regular store,
>> satisfying
>> >>>>>> all gurantees needed for further groupby / join. The Windowed
>> store is
>> >>>>>>
>> >>>>> not
>> >>>>
>> >>>>> keeping the values, so for the next statefull operation we would
>> >>>>>> need to instantiate an extra store. or we have the window store
>> also
>> >>>>>>
>> >>>>> have
>> >>>>
>> >>>>> the values then.
>> >>>>>>
>> >>>>>> Long story short. if we can flip in a custom group by before
>> >>>>>> repartitioning to the original primary key i think it would help
>> the
>> >>>>>>
>> >>>>> users
>> >>>>
>> >>>>> big time in building efficient apps. Given the original primary key
>> >>>>>>
>> >>>>> issue I
>> >>>>
>> >>>>> understand that we do not have a solid foundation to build on.
>> >>>>>> Leaving primary key carry along to the user. very unfortunate. I
>> could
>> >>>>>> understand the decision goes like that. I do not think its a good
>> >>>>>>
>> >>>>> decision.
>> >>>>
>> >>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> Thanks
>> >>>>>>> Adam
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>> >>>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
>> >>>>>>>
>> >>>>>> wrote:
>> >>>>
>> >>>>>
>> >>>>>>>       please remove me from this group
>> >>>>>>>
>> >>>>>>>       On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>> >>>>>>>       <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>> >>
>> >>>>>>>
>> >>>>>>>       wrote:
>> >>>>>>>
>> >>>>>>>       > Hi Adam,
>> >>>>>>>       >
>> >>>>>>>       > give me some time, will make such a chart. last time i
>> didn't
>> >>>>>>>       get along
>> >>>>>>>       > well with giphy and ruined all your charts.
>> >>>>>>>       > Hopefully i can get it done today
>> >>>>>>>       >
>> >>>>>>>       > On 08.09.2018 16:00, Adam Bellemare wrote:
>> >>>>>>>       > > Hi Jan
>> >>>>>>>       > >
>> >>>>>>>       > > I have included a diagram of what I attempted on the
>> KIP.
>> >>>>>>>       > >
>> >>>>>>>       >
>> >>>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>> >>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>> >>>>>>> inKTable-GroupBy+Reduce/Aggregate
>> >>>>>>>       <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>> >>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>> >>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>> >>>>>>>       > >
>> >>>>>>>       > > I attempted this back at the start of my own
>> implementation
>> >>>>>>> of
>> >>>>>>>       this
>> >>>>>>>       > > solution, and since I could not get it to work I have
>> since
>> >>>>>>>       discarded the
>> >>>>>>>       > > code. At this point in time, if you wish to continue
>> pursuing
>> >>>>>>>       for your
>> >>>>>>>       > > groupBy solution, I ask that you please create a
>> diagram on
>> >>>>>>>       the KIP
>> >>>>>>>       > > carefully explaining your solution. Please feel free to
>> use
>> >>>>>>>       the image I
>> >>>>>>>       > > just posted as a starting point. I am having trouble
>> >>>>>>>       understanding your
>> >>>>>>>       > > explanations but I think that a carefully constructed
>> diagram
>> >>>>>>>       will clear
>> >>>>>>>       > up
>> >>>>>>>       > > any misunderstandings. Alternately, please post a
>> >>>>>>>       comprehensive PR with
>> >>>>>>>       > > your solution. I can only guess at what you mean, and
>> since I
>> >>>>>>>       value my
>> >>>>>>>       > own
>> >>>>>>>       > > time as much as you value yours, I believe it is your
>> >>>>>>>       responsibility to
>> >>>>>>>       > > provide an implementation instead of me trying to guess.
>> >>>>>>>       > >
>> >>>>>>>       > > Adam
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > >
>> >>>>>>>       > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>> >>>>>>>       <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>> >>
>> >>>>>>>
>> >>>>>>>       > > wrote:
>> >>>>>>>       > >
>> >>>>>>>       > >> Hi James,
>> >>>>>>>       > >>
>> >>>>>>>       > >> nice to see you beeing interested. kafka streams at
>> this
>> >>>>>>>       point supports
>> >>>>>>>       > >> all sorts of joins as long as both streams have the
>> same
>> >>>>>>> key.
>> >>>>>>>       > >> Adam is currently implementing a join where a KTable
>> and a
>> >>>>>>>       KTable can
>> >>>>>>>       > have
>> >>>>>>>       > >> a one to many relation ship (1:n). We exploit that
>> rocksdb
>> >>>>>>> is
>> >>>>>>>
>> >>>>>> a
>> >>>>
>> >>>>>       > >> datastore that keeps data sorted (At least exposes an
>> API to
>> >>>>>>>       access the
>> >>>>>>>       > >> stored data in a sorted fashion).
>> >>>>>>>       > >>
>> >>>>>>>       > >> I think the technical caveats are well understood now
>> and we
>> >>>>>>>
>> >>>>>> are
>> >>>>
>> >>>>>       > basically
>> >>>>>>>       > >> down to philosophy and API Design ( when Adam sees my
>> newest
>> >>>>>>>       message).
>> >>>>>>>       > >> I have a lengthy track record of loosing those kinda
>> >>>>>>>       arguments within
>> >>>>>>>       > the
>> >>>>>>>       > >> streams community and I have no clue why. So I
>> literally
>> >>>>>>>       can't wait for
>> >>>>>>>       > you
>> >>>>>>>       > >> to churn through this thread and give you opinion on
>> how we
>> >>>>>>>       should
>> >>>>>>>       > design
>> >>>>>>>       > >> the return type of the oneToManyJoin and how many
>> power we
>> >>>>>>>       want to give
>> >>>>>>>       > to
>> >>>>>>>       > >> the user vs "simplicity" (where simplicity isn't
>> really that
>> >>>>>>>       as users
>> >>>>>>>       > still
>> >>>>>>>       > >> need to understand it I argue)
>> >>>>>>>       > >>
>> >>>>>>>       > >> waiting for you to join in on the discussion
>> >>>>>>>       > >>
>> >>>>>>>       > >> Best Jan
>> >>>>>>>       > >>
>> >>>>>>>       > >>
>> >>>>>>>       > >>
>> >>>>>>>       > >> On 07.09.2018 15:49, James Kwan wrote:
>> >>>>>>>       > >>
>> >>>>>>>       > >>> I am new to this group and I found this subject
>> >>>>>>>       interesting.  Sounds
>> >>>>>>>       > like
>> >>>>>>>       > >>> you guys want to implement a join table of two
>> streams? Is
>> >>>>>>> there
>> >>>>>>>       > somewhere
>> >>>>>>>       > >>> I can see the original requirement or proposal?
>> >>>>>>>       > >>>
>> >>>>>>>       > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>> >>>>>>>       <Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>> >>
>> >>>>>>>
>> >>>>>>>       > >>>> wrote:
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>> I'm currently testing using a Windowed Store to
>> store the
>> >>>>>>>       highwater
>> >>>>>>>       > >>>>> mark.
>> >>>>>>>       > >>>>> By all indications this should work fine, with the
>> caveat
>> >>>>>>>       being that
>> >>>>>>>       > it
>> >>>>>>>       > >>>>> can
>> >>>>>>>       > >>>>> only resolve out-of-order arrival for up to the
>> size of
>> >>>>>>>       the window
>> >>>>>>>       > (ie:
>> >>>>>>>       > >>>>> 24h, 72h, etc). This would remove the possibility
>> of it
>> >>>>>>>
>> >>>>>> being
>> >>>>
>> >>>>>       > unbounded
>> >>>>>>>       > >>>>> in
>> >>>>>>>       > >>>>> size.
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>> With regards to Jan's suggestion, I believe this is
>> where
>> >>>>>>>       we will
>> >>>>>>>       > have
>> >>>>>>>       > >>>>> to
>> >>>>>>>       > >>>>> remain in disagreement. While I do not disagree
>> with your
>> >>>>>>>       statement
>> >>>>>>>       > >>>>> about
>> >>>>>>>       > >>>>> there likely to be additional joins done in a
>> real-world
>> >>>>>>>       workflow, I
>> >>>>>>>       > do
>> >>>>>>>       > >>>>> not
>> >>>>>>>       > >>>>> see how you can conclusively deal with out-of-order
>> >>>>>>> arrival
>> >>>>>>> of
>> >>>>>>>       > >>>>> foreign-key
>> >>>>>>>       > >>>>> changes and subsequent joins. I have attempted what
>> I
>> >>>>>>>       think you have
>> >>>>>>>       > >>>>> proposed (without a high-water, using groupBy and
>> reduce)
>> >>>>>>>       and found
>> >>>>>>>       > >>>>> that if
>> >>>>>>>       > >>>>> the foreign key changes too quickly, or the load on
>> a
>> >>>>>>>       stream thread
>> >>>>>>>       > is
>> >>>>>>>       > >>>>> too
>> >>>>>>>       > >>>>> high, the joined messages will arrive out-of-order
>> and be
>> >>>>>>>       incorrectly
>> >>>>>>>       > >>>>> propagated, such that an intermediate event is
>> >>>>>>> represented
>> >>>>>>>       as the
>> >>>>>>>       > final
>> >>>>>>>       > >>>>> event.
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>> Can you shed some light on your groupBy
>> implementation.
>> >>>>>>>       There must be
>> >>>>>>>       > >>>> some sort of flaw in it.
>> >>>>>>>       > >>>> I have a suspicion where it is, I would just like to
>> >>>>>>>       confirm. The idea
>> >>>>>>>       > >>>> is bullet proof and it must be
>> >>>>>>>       > >>>> an implementation mess up. I would like to clarify
>> before
>> >>>>>>>       we draw a
>> >>>>>>>       > >>>> conclusion.
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>    Repartitioning the scattered events back to their
>> >>>>>>>
>> >>>>>> original
>> >>>>
>> >>>>>       > >>>>> partitions is the only way I know how to conclusively
>> deal
>> >>>>>>>       with
>> >>>>>>>       > >>>>> out-of-order events in a given time frame, and to
>> ensure
>> >>>>>>>       that the
>> >>>>>>>       > data
>> >>>>>>>       > >>>>> is
>> >>>>>>>       > >>>>> eventually consistent with the input events.
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>> If you have some code to share that illustrates your
>> >>>>>>>       approach, I
>> >>>>>>>       > would
>> >>>>>>>       > >>>>> be
>> >>>>>>>       > >>>>> very grateful as it would remove any
>> misunderstandings
>> >>>>>>>       that I may
>> >>>>>>>       > have.
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>> ah okay you were looking for my code. I don't have
>> >>>>>>>       something easily
>> >>>>>>>       > >>>> readable here as its bloated with OO-patterns.
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>> its anyhow trivial:
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>> @Override
>> >>>>>>>       > >>>>      public T apply(K aggKey, V value, T aggregate)
>> >>>>>>>       > >>>>      {
>> >>>>>>>       > >>>>          Map<U, V> currentStateAsMap =
>> asMap(aggregate);
>> >>>>>>> <<
>> >>>>>>>       imaginary
>> >>>>>>>       > >>>>          U toModifyKey = mapper.apply(value);
>> >>>>>>>       > >>>>              << this is the place where people
>> actually
>> >>>>>>>       gonna have
>> >>>>>>>       > issues
>> >>>>>>>       > >>>> and why you probably couldn't do it. we would need
>> to find
>> >>>>>>>       a solution
>> >>>>>>>       > here.
>> >>>>>>>       > >>>> I didn't realize that yet.
>> >>>>>>>       > >>>>              << we propagate the field in the
>> joiner, so
>> >>>>>>>       that we can
>> >>>>>>>       > pick
>> >>>>>>>       > >>>> it up in an aggregate. Probably you have not thought
>> of
>> >>>>>>>       this in your
>> >>>>>>>       > >>>> approach right?
>> >>>>>>>       > >>>>              << I am very open to find a generic
>> solution
>> >>>>>>>       here. In my
>> >>>>>>>       > >>>> honest opinion this is broken in KTableImpl.GroupBy
>> that
>> >>>>>>> it
>> >>>>>>>       looses
>> >>>>>>>       > the keys
>> >>>>>>>       > >>>> and only maintains the aggregate key.
>> >>>>>>>       > >>>>              << I abstracted it away back then way
>> before
>> >>>>>>> i
>> >>>>>>> was
>> >>>>>>>       > thinking
>> >>>>>>>       > >>>> of oneToMany join. That is why I didn't realize its
>> >>>>>>>       significance here.
>> >>>>>>>       > >>>>              << Opinions?
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>          for (V m : current)
>> >>>>>>>       > >>>>          {
>> >>>>>>>       > >>>> currentStateAsMap.put(mapper.apply(m), m);
>> >>>>>>>       > >>>>          }
>> >>>>>>>       > >>>>          if (isAdder)
>> >>>>>>>       > >>>>          {
>> >>>>>>>       > >>>> currentStateAsMap.put(toModifyKey, value);
>> >>>>>>>       > >>>>          }
>> >>>>>>>       > >>>>          else
>> >>>>>>>       > >>>>          {
>> >>>>>>>       > >>>> currentStateAsMap.remove(toModifyKey);
>> >>>>>>>       > >>>> if(currentStateAsMap.isEmpty()){
>> >>>>>>>       > >>>>                  return null;
>> >>>>>>>       > >>>>              }
>> >>>>>>>       > >>>>          }
>> >>>>>>>       > >>>>          retrun asAggregateType(currentStateAsMap)
>> >>>>>>>       > >>>>      }
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>>
>> >>>>>>>       > >>>> Thanks,
>> >>>>>>>       > >>>>> Adam
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>> >>>>>>>       > Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
>> >>
>> >>>>>>>
>> >>>>>>>       > >>>>> wrote:
>> >>>>>>>       > >>>>>
>> >>>>>>>       > >>>>> Thanks Adam for bringing Matthias to speed!
>> >>>>>>>       > >>>>>> about the differences. I think re-keying back
>> should be
>> >>>>>>>       optional at
>> >>>>>>>       > >>>>>> best.
>> >>>>>>>       > >>>>>> I would say we return a KScatteredTable with
>> reshuffle()
>> >>>>>>>       returning
>> >>>>>>>       > >>>>>> KTable<originalKey,Joined> to make the backwards
>> >>>>>>>       repartitioning
>> >>>>>>>       > >>>>>> optional.
>> >>>>>>>       > >>>>>> I am also in a big favour of doing the out of order
>> >>>>>>>       processing using
>> >>>>>>>       > >>>>>> group
>> >>>>>>>       > >>>>>> by instead high water mark tracking.
>> >>>>>>>       > >>>>>> Just because unbounded growth is just scary + It
>> saves
>> >>>>>>> us
>> >>>>>>>       the header
>> >>>>>>>       > >>>>>> stuff.
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>> I think the abstraction of always repartitioning
>> back is
>> >>>>>>>       just not so
>> >>>>>>>       > >>>>>> strong. Like the work has been done before we
>> partition
>> >>>>>>>       back and
>> >>>>>>>       > >>>>>> grouping
>> >>>>>>>       > >>>>>> by something else afterwards is really common.
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>> >>>>>>>       > >>>>>>
>> >>>>>>>       > >>>>>> Hi Matthias
>> >>>>>>>       > >>>>>>> Thank you for your feedback, I do appreciate it!
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> While name spacing would be possible, it would
>> require
>> >>>>>>> to
>> >>>>>>>       > deserialize
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I
>> would
>> >>>>>>>       suggest to
>> >>>>>>>       > no
>> >>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
>> >>>>>>>
>> >>>>>> becomes a
>> >>>>
>> >>>>>       > problem in
>> >>>>>>>       > >>>>>>>> the future, we can still add name spacing later
>> on.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Agreed. I will go with using a reserved string
>> and
>> >>>>>>>       document it.
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> My main concern about the design it the type of
>> the
>> >>>>>>>       result KTable:
>> >>>>>>>       > If
>> >>>>>>>       > >>>>>>> I
>> >>>>>>>       > >>>>>>> understood the proposal correctly,
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> In your example, you have table1 and table2
>> swapped.
>> >>>>>>>       Here is how it
>> >>>>>>>       > >>>>>>> works
>> >>>>>>>       > >>>>>>> currently:
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> 1) table1 has the records that contain the
>> foreign key
>> >>>>>>>       within their
>> >>>>>>>       > >>>>>>> value.
>> >>>>>>>       > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>> >>>>>>> <b,(fk=A,bar=2)>,
>> >>>>>>>       > >>>>>>> <c,(fk=B,bar=3)>
>> >>>>>>>       > >>>>>>> table2 input stream: <A,X>, <B,Y>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> 2) A Value mapper is required to extract the
>> foreign
>> >>>>>>> key.
>> >>>>>>>       > >>>>>>> table1 foreign key mapper: ( value => value.fk
>> >>>>>>>       <http://value.fk> )
>> >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> The mapper is applied to each element in table1,
>> and a
>> >>>>>>>       new combined
>> >>>>>>>       > >>>>>>> key is
>> >>>>>>>       > >>>>>>> made:
>> >>>>>>>       > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>> >>>>>>> (fk=A,bar=2)>,
>> >>>>>>>       <B-c,
>> >>>>>>>       > >>>>>>> (fk=B,bar=3)>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> 3) The rekeyed events are copartitioned with
>> table2:
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> a) Stream Thread with Partition 0:
>> >>>>>>>       > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>> >>>>>>>       (fk=A,bar=2)>
>> >>>>>>>       > >>>>>>> Table2: <A,X>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> b) Stream Thread with Partition 1:
>> >>>>>>>       > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>> >>>>>>>       > >>>>>>> Table2: <B,Y>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> 4) From here, they can be joined together locally
>> by
>> >>>>>>>       applying the
>> >>>>>>>       > >>>>>>> joiner
>> >>>>>>>       > >>>>>>> function.
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> At this point, Jan's design and my design
>> deviate. My
>> >>>>>>>       design goes
>> >>>>>>>       > on
>> >>>>>>>       > >>>>>>> to
>> >>>>>>>       > >>>>>>> repartition the data post-join and resolve
>> out-of-order
>> >>>>>>>       arrival of
>> >>>>>>>       > >>>>>>> records,
>> >>>>>>>       > >>>>>>> finally returning the data keyed just the
>> original key.
>> >>>>>>>       I do not
>> >>>>>>>       > >>>>>>> expose
>> >>>>>>>       > >>>>>>> the
>> >>>>>>>       > >>>>>>> CombinedKey or any of the internals outside of the
>> >>>>>>>       joinOnForeignKey
>> >>>>>>>       > >>>>>>> function. This does make for larger footprint,
>> but it
>> >>>>>>>       removes all
>> >>>>>>>       > >>>>>>> agency
>> >>>>>>>       > >>>>>>> for resolving out-of-order arrivals and handling
>> >>>>>>>       CombinedKeys from
>> >>>>>>>       > the
>> >>>>>>>       > >>>>>>> user. I believe that this makes the function much
>> >>>>>>> easier
>> >>>>>>>       to use.
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> Let me know if this helps resolve your questions,
>> and
>> >>>>>>>       please feel
>> >>>>>>>       > >>>>>>> free to
>> >>>>>>>       > >>>>>>> add anything else on your mind.
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> Thanks again,
>> >>>>>>>       > >>>>>>> Adam
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>> >>>>>>>       > >>>>>>> matthias@confluent.io <mailto:
>> matthias@confluent.io>>
>> >>>>>>>
>> >>>>>>>       > >>>>>>> wrote:
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>> Hi,
>> >>>>>>>       > >>>>>>>
>> >>>>>>>       > >>>>>>>> I am just catching up on this thread. I did not
>> read
>> >>>>>>>       everything so
>> >>>>>>>       > >>>>>>>> far,
>> >>>>>>>       > >>>>>>>> but want to share couple of initial thoughts:
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Headers: I think there is a fundamental
>> difference
>> >>>>>>>       between header
>> >>>>>>>       > >>>>>>>> usage
>> >>>>>>>       > >>>>>>>> in this KIP and KP-258. For 258, we add headers
>> to
>> >>>>>>>       changelog topic
>> >>>>>>>       > >>>>>>>> that
>> >>>>>>>       > >>>>>>>> are owned by Kafka Streams and nobody else is
>> supposed
>> >>>>>>>       to write
>> >>>>>>>       > into
>> >>>>>>>       > >>>>>>>> them. In fact, no user header are written into
>> the
>> >>>>>>>       changelog topic
>> >>>>>>>       > >>>>>>>> and
>> >>>>>>>       > >>>>>>>> thus, there are not conflicts.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Nevertheless, I don't see a big issue with using
>> >>>>>>>       headers within
>> >>>>>>>       > >>>>>>>> Streams.
>> >>>>>>>       > >>>>>>>> As long as we document it, we can have some
>> "reserved"
>> >>>>>>>       header keys
>> >>>>>>>       > >>>>>>>> and
>> >>>>>>>       > >>>>>>>> users are not allowed to use when processing
>> data with
>> >>>>>>>       Kafka
>> >>>>>>>       > Streams.
>> >>>>>>>       > >>>>>>>> IMHO, this should be ok.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> I think there is a safe way to avoid conflicts,
>> since
>> >>>>>>> these
>> >>>>>>>       > headers
>> >>>>>>>       > >>>>>>>> are
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>> only needed in internal topics (I think):
>> >>>>>>>       > >>>>>>>>> For internal and changelog topics, we can
>> namespace
>> >>>>>>>       all headers:
>> >>>>>>>       > >>>>>>>>> * user-defined headers are namespaced as
>> "external."
>> >>>>>>> +
>> >>>>>>>       headerKey
>> >>>>>>>       > >>>>>>>>> * internal headers are namespaced as
>> "internal." +
>> >>>>>>>       headerKey
>> >>>>>>>       > >>>>>>>>>
>> >>>>>>>       > >>>>>>>>> While name spacing would be possible, it would
>> >>>>>>> require
>> >>>>>>>
>> >>>>>> to
>> >>>>
>> >>>>>       > >>>>>>>> deserialize
>> >>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I
>> would
>> >>>>>>>       suggest to
>> >>>>>>>       > no
>> >>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
>> >>>>>>>
>> >>>>>> becomes a
>> >>>>
>> >>>>>       > problem in
>> >>>>>>>       > >>>>>>>> the future, we can still add name spacing later
>> on.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> My main concern about the design it the type of
>> the
>> >>>>>>>       result KTable:
>> >>>>>>>       > >>>>>>>> If I
>> >>>>>>>       > >>>>>>>> understood the proposal correctly,
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> KTable<K1,V1> table1 = ...
>> >>>>>>>       > >>>>>>>> KTable<K2,V2> table2 = ...
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> KTable<K1,V3> joinedTable =
>> table1.join(table2,...);
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> implies that the `joinedTable` has the same key
>> as the
>> >>>>>>>       left input
>> >>>>>>>       > >>>>>>>> table.
>> >>>>>>>       > >>>>>>>> IMHO, this does not work because if table2
>> contains
>> >>>>>>>       multiple rows
>> >>>>>>>       > >>>>>>>> that
>> >>>>>>>       > >>>>>>>> join with a record in table1 (what is the main
>> purpose
>> >>>>>>>
>> >>>>>> of
>> >>>>
>> >>>>> a
>> >>>>>>>       > foreign
>> >>>>>>>       > >>>>>>>> key
>> >>>>>>>       > >>>>>>>> join), the result table would only contain a
>> single
>> >>>>>>>       join result,
>> >>>>>>>       > but
>> >>>>>>>       > >>>>>>>> not
>> >>>>>>>       > >>>>>>>> multiple.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Example:
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> table1 input stream: <A,X>
>> >>>>>>>       > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> We use table2 value a foreign key to table1 key
>> (ie,
>> >>>>>>>       "A" joins).
>> >>>>>>>       > If
>> >>>>>>>       > >>>>>>>> the
>> >>>>>>>       > >>>>>>>> result key is the same key as key of table1, this
>> >>>>>>>       implies that the
>> >>>>>>>       > >>>>>>>> result can either be <A, join(X,1)> or <A,
>> join(X,2)>
>> >>>>>>>       but not
>> >>>>>>>       > both.
>> >>>>>>>       > >>>>>>>> Because the share the same key, whatever result
>> record
>> >>>>>>>       we emit
>> >>>>>>>       > later,
>> >>>>>>>       > >>>>>>>> overwrite the previous result.
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> This is the reason why Jan originally proposed
>> to use
>> >>>>>>> a
>> >>>>>>>       > combination
>> >>>>>>>       > >>>>>>>> of
>> >>>>>>>       > >>>>>>>> both primary keys of the input tables as key of
>> the
>> >>>>>>>       output table.
>> >>>>>>>       > >>>>>>>> This
>> >>>>>>>       > >>>>>>>> makes the keys of the output table unique and we
>> can
>> >>>>>>>       store both in
>> >>>>>>>       > >>>>>>>> the
>> >>>>>>>       > >>>>>>>> output table:
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b,
>> join(X,2)>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Thoughts?
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> -Matthias
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>> >>>>>>>       > >>>>>>>>
>> >>>>>>>       > >>>>>>>> Just on remark here.
>> >>>>>>>       > >>>>>>>>> The high-watermark could be disregarded. The
>> decision
>> >>>>>>>       about the
>> >>>>>>>       > >>>>>>>>> forward
>> >>>>>>>       > >>>>>>>>> depends on the size of the aggregated map.
>> >>>>>>>       > >>>>>>>>> Only 1 element long maps would be unpacked and
>> >>>>>>>       forwarded. 0
>> >>>>>>>       > element
>> >>>>>>>       > >>>>>>>>> maps
>> >>>>>>>       > >>>>>>>>> would be published as delete. Any other count
>> >>>>>>>       > >>>>>>>>> of map entries is in "waiting for correct
>> deletes to
>> >>>>>>>       > arrive"-state.
>> >>>>>>>       > >>>>>>>>>
>> >>>>>>>       > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>> >>>>>>>       > >>>>>>>>>
>> >>>>>>>       > >>>>>>>>> It does look like I could replace the second
>> >>>>>>>       repartition store
>> >>>>>>>       > and
>> >>>>>>>       > >>>>>>>>>> highwater store with a groupBy and reduce.
>> However,
>> >>>>>>>       it looks
>> >>>>>>>       > like
>> >>>>>>>       > >>>>>>>>>> I
>> >>>>>>>       > >>>>>>>>>> would
>> >>>>>>>       > >>>>>>>>>> still need to store the highwater value within
>> the
>> >>>>>>>       materialized
>> >>>>>>>       > >>>>>>>>>> store,
>> >>>>>>>       > >>>>>>>>>>
>> >>>>>>>       > >>>>>>>>>> to
>> >>>>>>>       > >>>>>>>>> compare the arrival of out-of-order records
>> (assuming
>> >>>>>>>
>> >>>>>> my
>> >>>>
>> >>>>>       > >>>>>>>>> understanding
>> >>>>>>>       > >>>>>>>>> of
>> >>>>>>>       > >>>>>>>>> THIS is correct...). This in effect is the same
>> as
>> >>>>>>> the
>> >>>>>>>       design I
>> >>>>>>>       > have
>> >>>>>>>       > >>>>>>>>> now,
>> >>>>>>>       > >>>>>>>>> just with the two tables merged together.
>> >>>>>>>       > >>>>>>>>>
>> >>>>>>>       >
>> >>>>>>>       >
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>> --
>> >>>>> -- Guozhang
>> >>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>> --
>> >>>> -- Guozhang
>> >>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Jan

With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you
actually only need to specify the amount of segments you want and how large
they are. To the best of my understanding, what happens is that the
segments are automatically rolled over as new data with new timestamps are
created. We use this exact functionality in some of the work done
internally at my company. For reference, this is the hopping windowed store.
https://kafka.apache.org/11/documentation/streams/developer-guide/dsl-api.html#id21

In the code that I have provided, there are going to be two 24h segments.
When a record is put into the windowStore, it will be inserted at time T in
both segments. The two segments will always overlap by 12h. As time goes on
and new records are added (say at time T+12h+), the oldest segment will be
automatically deleted and a new segment created. The records are by default
inserted with the context.timestamp(), such that it is the record time, not
the clock time, which is used.

To the best of my understanding, the timestamps are retained when restoring
from the changelog.

Basically, this is heavy-handed way to deal with TTL at a segment-level,
instead of at an individual record level.

On Tue, Sep 25, 2018 at 5:18 PM Jan Filipiak <Ja...@trivago.com>
wrote:

> Will that work? I expected it to blow up with ClassCastException or
> similar.
>
> You either would have to specify the window you fetch/put or iterate
> across all windows the key was found in right?
>
> I just hope the window-store doesn't check stream-time under the hoods
> that would be a questionable interface.
>
> If it does: did you see my comment on checking all the windows earlier?
> that would be needed to actually give reasonable time gurantees.
>
> Best
>
>
>
> On 25.09.2018 13:18, Adam Bellemare wrote:
> > Hi Jan
> >
> > Check for  " highwaterMat " in the PR. I only changed the state store,
> not
> > the ProcessorSupplier.
> >
> > Thanks,
> > Adam
> >
> > On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >>
> >>
> >> On 24.09.2018 16:26, Adam Bellemare wrote:
> >>
> >>> @Guozhang
> >>>
> >>> Thanks for the information. This is indeed something that will be
> >>> extremely
> >>> useful for this KIP.
> >>>
> >>> @Jan
> >>> Thanks for your explanations. That being said, I will not be moving
> ahead
> >>> with an implementation using reshuffle/groupBy solution as you propose.
> >>> That being said, if you wish to implement it yourself off of my
> current PR
> >>> and submit it as a competitive alternative, I would be more than happy
> to
> >>> help vet that as an alternate solution. As it stands right now, I do
> not
> >>> really have more time to invest into alternatives without there being a
> >>> strong indication from the binding voters which they would prefer.
> >>>
> >>>
> >> Hey, total no worries. I think I personally gave up on the streams DSL
> for
> >> some time already, otherwise I would have pulled this KIP through
> already.
> >> I am currently reimplementing my own DSL based on PAPI.
> >>
> >>
> >>> I will look at finishing up my PR with the windowed state store in the
> >>> next
> >>> week or so, exercising it via tests, and then I will come back for
> final
> >>> discussions. In the meantime, I hope that any of the binding voters
> could
> >>> take a look at the KIP in the wiki. I have updated it according to the
> >>> latest plan:
> >>>
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
> >>> Support+non-key+joining+in+KTable
> >>>
> >>> I have also updated the KIP PR to use a windowed store. This could be
> >>> replaced by the results of KIP-258 whenever they are completed.
> >>> https://github.com/apache/kafka/pull/5527
> >>>
> >>> Thanks,
> >>>
> >>> Adam
> >>>
> >>
> >> Is the HighWatermarkResolverProccessorsupplier already updated in the
> PR?
> >> expected it to change to Windowed<K>,Long Missing something?
> >>
> >>
> >>
> >>>
> >>>
> >>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
> >>> wrote:
> >>>
> >>> Correction on my previous email: KAFKA-5533 is the wrong link, as it is
> >>>> for
> >>>> corresponding changelog mechanisms. But as part of KIP-258 we do want
> to
> >>>> have "handling out-of-order data for source KTable" such that instead
> of
> >>>> blindly apply the updates to the materialized store, i.e. following
> >>>> offset
> >>>> ordering, we will reject updates that are older than the current key's
> >>>> timestamps, i.e. following timestamp ordering.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
> >>>> wrote:
> >>>>
> >>>> Hello Adam,
> >>>>>
> >>>>> Thanks for the explanation. Regarding the final step (i.e. the high
> >>>>> watermark store, now altered to be replaced with a window store), I
> >>>>> think
> >>>>> another current on-going KIP may actually help:
> >>>>>
> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >>>>>
> >>>>>
> >>>>> This is for adding the timestamp into a key-value store (i.e. only
> for
> >>>>> non-windowed KTable), and then one of its usage, as described in
> >>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can
> then
> >>>>> "reject" updates from the source topics if its timestamp is smaller
> than
> >>>>> the current key's latest update timestamp. I think it is very
> similar to
> >>>>> what you have in mind for high watermark based filtering, while you
> only
> >>>>> need to make sure that the timestamps of the joining records are
> >>>>>
> >>>> correctly
> >>>>
> >>>>> inherited though the whole topology to the final stage.
> >>>>>
> >>>>> Note that this KIP is for key-value store and hence non-windowed
> KTables
> >>>>> only, but for windowed KTables we do not really have a good support
> for
> >>>>> their joins anyways (
> https://issues.apache.org/jira/browse/KAFKA-7107)
> >>>>> I
> >>>>> think we can just consider non-windowed KTable-KTable non-key joins
> for
> >>>>> now. In which case, KIP-258 should help.
> >>>>>
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <
> Jan.Filipiak@trivago.com
> >>>>>>
> >>>>> wrote:
> >>>>>
> >>>>>
> >>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>>>>>
> >>>>>> Hi Guozhang
> >>>>>>>
> >>>>>>> Current highwater mark implementation would grow endlessly based on
> >>>>>>> primary key of original event. It is a pair of (<this table primary
> >>>>>>>
> >>>>>> key>,
> >>>>
> >>>>> <highest offset seen for that key>). This is used to differentiate
> >>>>>>>
> >>>>>> between
> >>>>
> >>>>> late arrivals and new updates. My newest proposal would be to replace
> >>>>>>>
> >>>>>> it
> >>>>
> >>>>> with a Windowed state store of Duration N. This would allow the same
> >>>>>>> behaviour, but cap the size based on time. This should allow for
> all
> >>>>>>> late-arriving events to be processed, and should be customizable by
> >>>>>>> the
> >>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
> >>>>>>>
> >>>>>> window,
> >>>>
> >>>>> or perhaps 7 days...).
> >>>>>>>
> >>>>>>> Hi Adam, using time based retention can do the trick here. Even if
> I
> >>>>>> would still like to see the automatic repartitioning optional since
> I
> >>>>>>
> >>>>> would
> >>>>
> >>>>> just reshuffle again. With windowed store I am a little bit sceptical
> >>>>>>
> >>>>> about
> >>>>
> >>>>> how to determine the window. So esentially one could run into
> problems
> >>>>>>
> >>>>> when
> >>>>
> >>>>> the rapid change happens near a window border. I will check you
> >>>>>> implementation in detail, if its problematic, we could still check
> >>>>>> _all_
> >>>>>> windows on read with not to bad performance impact I guess. Will let
> >>>>>> you
> >>>>>> know if the implementation would be correct as is. I wouldn't not
> like
> >>>>>>
> >>>>> to
> >>>>
> >>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I
> >>>>>>
> >>>>> think
> >>>>
> >>>>> we can't expect that.
> >>>>>>
> >>>>>>
> >>>>>>>
> >>>>>>> @Jan
> >>>>>>> I believe I understand what you mean now - thanks for the diagram,
> it
> >>>>>>> did really help. You are correct that I do not have the original
> >>>>>>>
> >>>>>> primary
> >>>>
> >>>>> key available, and I can see that if it was available then you would
> be
> >>>>>>> able to add and remove events from the Map. That being said, I
> >>>>>>>
> >>>>>> encourage
> >>>>
> >>>>> you to finish your diagrams / charts just for clarity for everyone
> >>>>>>>
> >>>>>> else.
> >>>>
> >>>>>
> >>>>>>> Yeah 100%, this giphy thing is just really hard work. But I
> understand
> >>>>>>>
> >>>>>> the benefits for the rest. Sorry about the original primary key, We
> >>>>>> have
> >>>>>> join and Group by implemented our own in PAPI and basically not
> using
> >>>>>>
> >>>>> any
> >>>>
> >>>>> DSL (Just the abstraction). Completely missed that in original DSL
> its
> >>>>>>
> >>>>> not
> >>>>
> >>>>> there and just assumed it. total brain mess up on my end. Will finish
> >>>>>>
> >>>>> the
> >>>>
> >>>>> chart as soon as i get a quite evening this week.
> >>>>>>
> >>>>>> My follow up question for you is, won't the Map stay inside the
> State
> >>>>>>
> >>>>>>> Store indefinitely after all of the changes have propagated? Isn't
> >>>>>>> this
> >>>>>>> effectively the same as a highwater mark state store?
> >>>>>>>
> >>>>>>> Thing is that if the map is empty, substractor is gonna return
> `null`
> >>>>>>
> >>>>> and
> >>>>
> >>>>> the key is removed from the keyspace. But there is going to be a
> store
> >>>>>> 100%, the good thing is that I can use this store directly for
> >>>>>> materialize() / enableSendingOldValues() is a regular store,
> satisfying
> >>>>>> all gurantees needed for further groupby / join. The Windowed store
> is
> >>>>>>
> >>>>> not
> >>>>
> >>>>> keeping the values, so for the next statefull operation we would
> >>>>>> need to instantiate an extra store. or we have the window store also
> >>>>>>
> >>>>> have
> >>>>
> >>>>> the values then.
> >>>>>>
> >>>>>> Long story short. if we can flip in a custom group by before
> >>>>>> repartitioning to the original primary key i think it would help the
> >>>>>>
> >>>>> users
> >>>>
> >>>>> big time in building efficient apps. Given the original primary key
> >>>>>>
> >>>>> issue I
> >>>>
> >>>>> understand that we do not have a solid foundation to build on.
> >>>>>> Leaving primary key carry along to the user. very unfortunate. I
> could
> >>>>>> understand the decision goes like that. I do not think its a good
> >>>>>>
> >>>>> decision.
> >>>>
> >>>>>
> >>>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> Thanks
> >>>>>>> Adam
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
> >>>>>>>
> >>>>>> wrote:
> >>>>
> >>>>>
> >>>>>>>       please remove me from this group
> >>>>>>>
> >>>>>>>       On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>>>>>
> >>>>>>>       wrote:
> >>>>>>>
> >>>>>>>       > Hi Adam,
> >>>>>>>       >
> >>>>>>>       > give me some time, will make such a chart. last time i
> didn't
> >>>>>>>       get along
> >>>>>>>       > well with giphy and ruined all your charts.
> >>>>>>>       > Hopefully i can get it done today
> >>>>>>>       >
> >>>>>>>       > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>>>>>       > > Hi Jan
> >>>>>>>       > >
> >>>>>>>       > > I have included a diagram of what I attempted on the KIP.
> >>>>>>>       > >
> >>>>>>>       >
> >>>>>>>       https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>>>>>> inKTable-GroupBy+Reduce/Aggregate
> >>>>>>>       <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>>>>>       > >
> >>>>>>>       > > I attempted this back at the start of my own
> implementation
> >>>>>>> of
> >>>>>>>       this
> >>>>>>>       > > solution, and since I could not get it to work I have
> since
> >>>>>>>       discarded the
> >>>>>>>       > > code. At this point in time, if you wish to continue
> pursuing
> >>>>>>>       for your
> >>>>>>>       > > groupBy solution, I ask that you please create a diagram
> on
> >>>>>>>       the KIP
> >>>>>>>       > > carefully explaining your solution. Please feel free to
> use
> >>>>>>>       the image I
> >>>>>>>       > > just posted as a starting point. I am having trouble
> >>>>>>>       understanding your
> >>>>>>>       > > explanations but I think that a carefully constructed
> diagram
> >>>>>>>       will clear
> >>>>>>>       > up
> >>>>>>>       > > any misunderstandings. Alternately, please post a
> >>>>>>>       comprehensive PR with
> >>>>>>>       > > your solution. I can only guess at what you mean, and
> since I
> >>>>>>>       value my
> >>>>>>>       > own
> >>>>>>>       > > time as much as you value yours, I believe it is your
> >>>>>>>       responsibility to
> >>>>>>>       > > provide an implementation instead of me trying to guess.
> >>>>>>>       > >
> >>>>>>>       > > Adam
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > >
> >>>>>>>       > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>>>>>
> >>>>>>>       > > wrote:
> >>>>>>>       > >
> >>>>>>>       > >> Hi James,
> >>>>>>>       > >>
> >>>>>>>       > >> nice to see you beeing interested. kafka streams at this
> >>>>>>>       point supports
> >>>>>>>       > >> all sorts of joins as long as both streams have the same
> >>>>>>> key.
> >>>>>>>       > >> Adam is currently implementing a join where a KTable
> and a
> >>>>>>>       KTable can
> >>>>>>>       > have
> >>>>>>>       > >> a one to many relation ship (1:n). We exploit that
> rocksdb
> >>>>>>> is
> >>>>>>>
> >>>>>> a
> >>>>
> >>>>>       > >> datastore that keeps data sorted (At least exposes an API
> to
> >>>>>>>       access the
> >>>>>>>       > >> stored data in a sorted fashion).
> >>>>>>>       > >>
> >>>>>>>       > >> I think the technical caveats are well understood now
> and we
> >>>>>>>
> >>>>>> are
> >>>>
> >>>>>       > basically
> >>>>>>>       > >> down to philosophy and API Design ( when Adam sees my
> newest
> >>>>>>>       message).
> >>>>>>>       > >> I have a lengthy track record of loosing those kinda
> >>>>>>>       arguments within
> >>>>>>>       > the
> >>>>>>>       > >> streams community and I have no clue why. So I literally
> >>>>>>>       can't wait for
> >>>>>>>       > you
> >>>>>>>       > >> to churn through this thread and give you opinion on
> how we
> >>>>>>>       should
> >>>>>>>       > design
> >>>>>>>       > >> the return type of the oneToManyJoin and how many power
> we
> >>>>>>>       want to give
> >>>>>>>       > to
> >>>>>>>       > >> the user vs "simplicity" (where simplicity isn't really
> that
> >>>>>>>       as users
> >>>>>>>       > still
> >>>>>>>       > >> need to understand it I argue)
> >>>>>>>       > >>
> >>>>>>>       > >> waiting for you to join in on the discussion
> >>>>>>>       > >>
> >>>>>>>       > >> Best Jan
> >>>>>>>       > >>
> >>>>>>>       > >>
> >>>>>>>       > >>
> >>>>>>>       > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>>>>>       > >>
> >>>>>>>       > >>> I am new to this group and I found this subject
> >>>>>>>       interesting.  Sounds
> >>>>>>>       > like
> >>>>>>>       > >>> you guys want to implement a join table of two
> streams? Is
> >>>>>>> there
> >>>>>>>       > somewhere
> >>>>>>>       > >>> I can see the original requirement or proposal?
> >>>>>>>       > >>>
> >>>>>>>       > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>>>>>
> >>>>>>>       > >>>> wrote:
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>
> >>>>>>>       > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>> I'm currently testing using a Windowed Store to
> store the
> >>>>>>>       highwater
> >>>>>>>       > >>>>> mark.
> >>>>>>>       > >>>>> By all indications this should work fine, with the
> caveat
> >>>>>>>       being that
> >>>>>>>       > it
> >>>>>>>       > >>>>> can
> >>>>>>>       > >>>>> only resolve out-of-order arrival for up to the size
> of
> >>>>>>>       the window
> >>>>>>>       > (ie:
> >>>>>>>       > >>>>> 24h, 72h, etc). This would remove the possibility of
> it
> >>>>>>>
> >>>>>> being
> >>>>
> >>>>>       > unbounded
> >>>>>>>       > >>>>> in
> >>>>>>>       > >>>>> size.
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>> With regards to Jan's suggestion, I believe this is
> where
> >>>>>>>       we will
> >>>>>>>       > have
> >>>>>>>       > >>>>> to
> >>>>>>>       > >>>>> remain in disagreement. While I do not disagree with
> your
> >>>>>>>       statement
> >>>>>>>       > >>>>> about
> >>>>>>>       > >>>>> there likely to be additional joins done in a
> real-world
> >>>>>>>       workflow, I
> >>>>>>>       > do
> >>>>>>>       > >>>>> not
> >>>>>>>       > >>>>> see how you can conclusively deal with out-of-order
> >>>>>>> arrival
> >>>>>>> of
> >>>>>>>       > >>>>> foreign-key
> >>>>>>>       > >>>>> changes and subsequent joins. I have attempted what I
> >>>>>>>       think you have
> >>>>>>>       > >>>>> proposed (without a high-water, using groupBy and
> reduce)
> >>>>>>>       and found
> >>>>>>>       > >>>>> that if
> >>>>>>>       > >>>>> the foreign key changes too quickly, or the load on a
> >>>>>>>       stream thread
> >>>>>>>       > is
> >>>>>>>       > >>>>> too
> >>>>>>>       > >>>>> high, the joined messages will arrive out-of-order
> and be
> >>>>>>>       incorrectly
> >>>>>>>       > >>>>> propagated, such that an intermediate event is
> >>>>>>> represented
> >>>>>>>       as the
> >>>>>>>       > final
> >>>>>>>       > >>>>> event.
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>> Can you shed some light on your groupBy
> implementation.
> >>>>>>>       There must be
> >>>>>>>       > >>>> some sort of flaw in it.
> >>>>>>>       > >>>> I have a suspicion where it is, I would just like to
> >>>>>>>       confirm. The idea
> >>>>>>>       > >>>> is bullet proof and it must be
> >>>>>>>       > >>>> an implementation mess up. I would like to clarify
> before
> >>>>>>>       we draw a
> >>>>>>>       > >>>> conclusion.
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>    Repartitioning the scattered events back to their
> >>>>>>>
> >>>>>> original
> >>>>
> >>>>>       > >>>>> partitions is the only way I know how to conclusively
> deal
> >>>>>>>       with
> >>>>>>>       > >>>>> out-of-order events in a given time frame, and to
> ensure
> >>>>>>>       that the
> >>>>>>>       > data
> >>>>>>>       > >>>>> is
> >>>>>>>       > >>>>> eventually consistent with the input events.
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>> If you have some code to share that illustrates your
> >>>>>>>       approach, I
> >>>>>>>       > would
> >>>>>>>       > >>>>> be
> >>>>>>>       > >>>>> very grateful as it would remove any
> misunderstandings
> >>>>>>>       that I may
> >>>>>>>       > have.
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>> ah okay you were looking for my code. I don't have
> >>>>>>>       something easily
> >>>>>>>       > >>>> readable here as its bloated with OO-patterns.
> >>>>>>>       > >>>>
> >>>>>>>       > >>>> its anyhow trivial:
> >>>>>>>       > >>>>
> >>>>>>>       > >>>> @Override
> >>>>>>>       > >>>>      public T apply(K aggKey, V value, T aggregate)
> >>>>>>>       > >>>>      {
> >>>>>>>       > >>>>          Map<U, V> currentStateAsMap =
> asMap(aggregate);
> >>>>>>> <<
> >>>>>>>       imaginary
> >>>>>>>       > >>>>          U toModifyKey = mapper.apply(value);
> >>>>>>>       > >>>>              << this is the place where people
> actually
> >>>>>>>       gonna have
> >>>>>>>       > issues
> >>>>>>>       > >>>> and why you probably couldn't do it. we would need to
> find
> >>>>>>>       a solution
> >>>>>>>       > here.
> >>>>>>>       > >>>> I didn't realize that yet.
> >>>>>>>       > >>>>              << we propagate the field in the joiner,
> so
> >>>>>>>       that we can
> >>>>>>>       > pick
> >>>>>>>       > >>>> it up in an aggregate. Probably you have not thought
> of
> >>>>>>>       this in your
> >>>>>>>       > >>>> approach right?
> >>>>>>>       > >>>>              << I am very open to find a generic
> solution
> >>>>>>>       here. In my
> >>>>>>>       > >>>> honest opinion this is broken in KTableImpl.GroupBy
> that
> >>>>>>> it
> >>>>>>>       looses
> >>>>>>>       > the keys
> >>>>>>>       > >>>> and only maintains the aggregate key.
> >>>>>>>       > >>>>              << I abstracted it away back then way
> before
> >>>>>>> i
> >>>>>>> was
> >>>>>>>       > thinking
> >>>>>>>       > >>>> of oneToMany join. That is why I didn't realize its
> >>>>>>>       significance here.
> >>>>>>>       > >>>>              << Opinions?
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>          for (V m : current)
> >>>>>>>       > >>>>          {
> >>>>>>>       > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >>>>>>>       > >>>>          }
> >>>>>>>       > >>>>          if (isAdder)
> >>>>>>>       > >>>>          {
> >>>>>>>       > >>>> currentStateAsMap.put(toModifyKey, value);
> >>>>>>>       > >>>>          }
> >>>>>>>       > >>>>          else
> >>>>>>>       > >>>>          {
> >>>>>>>       > >>>> currentStateAsMap.remove(toModifyKey);
> >>>>>>>       > >>>> if(currentStateAsMap.isEmpty()){
> >>>>>>>       > >>>>                  return null;
> >>>>>>>       > >>>>              }
> >>>>>>>       > >>>>          }
> >>>>>>>       > >>>>          retrun asAggregateType(currentStateAsMap)
> >>>>>>>       > >>>>      }
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>
> >>>>>>>       > >>>>
> >>>>>>>       > >>>> Thanks,
> >>>>>>>       > >>>>> Adam
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> >>>>>>>       > Jan.Filipiak@trivago.com <mailto:Jan.Filipiak@trivago.com
> >>
> >>>>>>>
> >>>>>>>       > >>>>> wrote:
> >>>>>>>       > >>>>>
> >>>>>>>       > >>>>> Thanks Adam for bringing Matthias to speed!
> >>>>>>>       > >>>>>> about the differences. I think re-keying back
> should be
> >>>>>>>       optional at
> >>>>>>>       > >>>>>> best.
> >>>>>>>       > >>>>>> I would say we return a KScatteredTable with
> reshuffle()
> >>>>>>>       returning
> >>>>>>>       > >>>>>> KTable<originalKey,Joined> to make the backwards
> >>>>>>>       repartitioning
> >>>>>>>       > >>>>>> optional.
> >>>>>>>       > >>>>>> I am also in a big favour of doing the out of order
> >>>>>>>       processing using
> >>>>>>>       > >>>>>> group
> >>>>>>>       > >>>>>> by instead high water mark tracking.
> >>>>>>>       > >>>>>> Just because unbounded growth is just scary + It
> saves
> >>>>>>> us
> >>>>>>>       the header
> >>>>>>>       > >>>>>> stuff.
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>> I think the abstraction of always repartitioning
> back is
> >>>>>>>       just not so
> >>>>>>>       > >>>>>> strong. Like the work has been done before we
> partition
> >>>>>>>       back and
> >>>>>>>       > >>>>>> grouping
> >>>>>>>       > >>>>>> by something else afterwards is really common.
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>>>>>       > >>>>>>
> >>>>>>>       > >>>>>> Hi Matthias
> >>>>>>>       > >>>>>>> Thank you for your feedback, I do appreciate it!
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> While name spacing would be possible, it would
> require
> >>>>>>> to
> >>>>>>>       > deserialize
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I
> would
> >>>>>>>       suggest to
> >>>>>>>       > no
> >>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
> >>>>>>>
> >>>>>> becomes a
> >>>>
> >>>>>       > problem in
> >>>>>>>       > >>>>>>>> the future, we can still add name spacing later
> on.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Agreed. I will go with using a reserved string and
> >>>>>>>       document it.
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> My main concern about the design it the type of the
> >>>>>>>       result KTable:
> >>>>>>>       > If
> >>>>>>>       > >>>>>>> I
> >>>>>>>       > >>>>>>> understood the proposal correctly,
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> In your example, you have table1 and table2
> swapped.
> >>>>>>>       Here is how it
> >>>>>>>       > >>>>>>> works
> >>>>>>>       > >>>>>>> currently:
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> 1) table1 has the records that contain the foreign
> key
> >>>>>>>       within their
> >>>>>>>       > >>>>>>> value.
> >>>>>>>       > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
> >>>>>>> <b,(fk=A,bar=2)>,
> >>>>>>>       > >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>>       > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> 2) A Value mapper is required to extract the
> foreign
> >>>>>>> key.
> >>>>>>>       > >>>>>>> table1 foreign key mapper: ( value => value.fk
> >>>>>>>       <http://value.fk> )
> >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> The mapper is applied to each element in table1,
> and a
> >>>>>>>       new combined
> >>>>>>>       > >>>>>>> key is
> >>>>>>>       > >>>>>>> made:
> >>>>>>>       > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
> >>>>>>> (fk=A,bar=2)>,
> >>>>>>>       <B-c,
> >>>>>>>       > >>>>>>> (fk=B,bar=3)>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> 3) The rekeyed events are copartitioned with
> table2:
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>>       > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
> >>>>>>>       (fk=A,bar=2)>
> >>>>>>>       > >>>>>>> Table2: <A,X>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>>       > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>>>>>       > >>>>>>> Table2: <B,Y>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> 4) From here, they can be joined together locally
> by
> >>>>>>>       applying the
> >>>>>>>       > >>>>>>> joiner
> >>>>>>>       > >>>>>>> function.
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> At this point, Jan's design and my design deviate.
> My
> >>>>>>>       design goes
> >>>>>>>       > on
> >>>>>>>       > >>>>>>> to
> >>>>>>>       > >>>>>>> repartition the data post-join and resolve
> out-of-order
> >>>>>>>       arrival of
> >>>>>>>       > >>>>>>> records,
> >>>>>>>       > >>>>>>> finally returning the data keyed just the original
> key.
> >>>>>>>       I do not
> >>>>>>>       > >>>>>>> expose
> >>>>>>>       > >>>>>>> the
> >>>>>>>       > >>>>>>> CombinedKey or any of the internals outside of the
> >>>>>>>       joinOnForeignKey
> >>>>>>>       > >>>>>>> function. This does make for larger footprint, but
> it
> >>>>>>>       removes all
> >>>>>>>       > >>>>>>> agency
> >>>>>>>       > >>>>>>> for resolving out-of-order arrivals and handling
> >>>>>>>       CombinedKeys from
> >>>>>>>       > the
> >>>>>>>       > >>>>>>> user. I believe that this makes the function much
> >>>>>>> easier
> >>>>>>>       to use.
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> Let me know if this helps resolve your questions,
> and
> >>>>>>>       please feel
> >>>>>>>       > >>>>>>> free to
> >>>>>>>       > >>>>>>> add anything else on your mind.
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> Thanks again,
> >>>>>>>       > >>>>>>> Adam
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> >>>>>>>       > >>>>>>> matthias@confluent.io <mailto:
> matthias@confluent.io>>
> >>>>>>>
> >>>>>>>       > >>>>>>> wrote:
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>> Hi,
> >>>>>>>       > >>>>>>>
> >>>>>>>       > >>>>>>>> I am just catching up on this thread. I did not
> read
> >>>>>>>       everything so
> >>>>>>>       > >>>>>>>> far,
> >>>>>>>       > >>>>>>>> but want to share couple of initial thoughts:
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Headers: I think there is a fundamental difference
> >>>>>>>       between header
> >>>>>>>       > >>>>>>>> usage
> >>>>>>>       > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
> >>>>>>>       changelog topic
> >>>>>>>       > >>>>>>>> that
> >>>>>>>       > >>>>>>>> are owned by Kafka Streams and nobody else is
> supposed
> >>>>>>>       to write
> >>>>>>>       > into
> >>>>>>>       > >>>>>>>> them. In fact, no user header are written into the
> >>>>>>>       changelog topic
> >>>>>>>       > >>>>>>>> and
> >>>>>>>       > >>>>>>>> thus, there are not conflicts.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Nevertheless, I don't see a big issue with using
> >>>>>>>       headers within
> >>>>>>>       > >>>>>>>> Streams.
> >>>>>>>       > >>>>>>>> As long as we document it, we can have some
> "reserved"
> >>>>>>>       header keys
> >>>>>>>       > >>>>>>>> and
> >>>>>>>       > >>>>>>>> users are not allowed to use when processing data
> with
> >>>>>>>       Kafka
> >>>>>>>       > Streams.
> >>>>>>>       > >>>>>>>> IMHO, this should be ok.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> I think there is a safe way to avoid conflicts,
> since
> >>>>>>> these
> >>>>>>>       > headers
> >>>>>>>       > >>>>>>>> are
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>> only needed in internal topics (I think):
> >>>>>>>       > >>>>>>>>> For internal and changelog topics, we can
> namespace
> >>>>>>>       all headers:
> >>>>>>>       > >>>>>>>>> * user-defined headers are namespaced as
> "external."
> >>>>>>> +
> >>>>>>>       headerKey
> >>>>>>>       > >>>>>>>>> * internal headers are namespaced as "internal."
> +
> >>>>>>>       headerKey
> >>>>>>>       > >>>>>>>>>
> >>>>>>>       > >>>>>>>>> While name spacing would be possible, it would
> >>>>>>> require
> >>>>>>>
> >>>>>> to
> >>>>
> >>>>>       > >>>>>>>> deserialize
> >>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I
> would
> >>>>>>>       suggest to
> >>>>>>>       > no
> >>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
> >>>>>>>
> >>>>>> becomes a
> >>>>
> >>>>>       > problem in
> >>>>>>>       > >>>>>>>> the future, we can still add name spacing later
> on.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> My main concern about the design it the type of
> the
> >>>>>>>       result KTable:
> >>>>>>>       > >>>>>>>> If I
> >>>>>>>       > >>>>>>>> understood the proposal correctly,
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>       > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> KTable<K1,V3> joinedTable =
> table1.join(table2,...);
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> implies that the `joinedTable` has the same key
> as the
> >>>>>>>       left input
> >>>>>>>       > >>>>>>>> table.
> >>>>>>>       > >>>>>>>> IMHO, this does not work because if table2
> contains
> >>>>>>>       multiple rows
> >>>>>>>       > >>>>>>>> that
> >>>>>>>       > >>>>>>>> join with a record in table1 (what is the main
> purpose
> >>>>>>>
> >>>>>> of
> >>>>
> >>>>> a
> >>>>>>>       > foreign
> >>>>>>>       > >>>>>>>> key
> >>>>>>>       > >>>>>>>> join), the result table would only contain a
> single
> >>>>>>>       join result,
> >>>>>>>       > but
> >>>>>>>       > >>>>>>>> not
> >>>>>>>       > >>>>>>>> multiple.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Example:
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> table1 input stream: <A,X>
> >>>>>>>       > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> We use table2 value a foreign key to table1 key
> (ie,
> >>>>>>>       "A" joins).
> >>>>>>>       > If
> >>>>>>>       > >>>>>>>> the
> >>>>>>>       > >>>>>>>> result key is the same key as key of table1, this
> >>>>>>>       implies that the
> >>>>>>>       > >>>>>>>> result can either be <A, join(X,1)> or <A,
> join(X,2)>
> >>>>>>>       but not
> >>>>>>>       > both.
> >>>>>>>       > >>>>>>>> Because the share the same key, whatever result
> record
> >>>>>>>       we emit
> >>>>>>>       > later,
> >>>>>>>       > >>>>>>>> overwrite the previous result.
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> This is the reason why Jan originally proposed to
> use
> >>>>>>> a
> >>>>>>>       > combination
> >>>>>>>       > >>>>>>>> of
> >>>>>>>       > >>>>>>>> both primary keys of the input tables as key of
> the
> >>>>>>>       output table.
> >>>>>>>       > >>>>>>>> This
> >>>>>>>       > >>>>>>>> makes the keys of the output table unique and we
> can
> >>>>>>>       store both in
> >>>>>>>       > >>>>>>>> the
> >>>>>>>       > >>>>>>>> output table:
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Thoughts?
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> -Matthias
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>>>       > >>>>>>>>
> >>>>>>>       > >>>>>>>> Just on remark here.
> >>>>>>>       > >>>>>>>>> The high-watermark could be disregarded. The
> decision
> >>>>>>>       about the
> >>>>>>>       > >>>>>>>>> forward
> >>>>>>>       > >>>>>>>>> depends on the size of the aggregated map.
> >>>>>>>       > >>>>>>>>> Only 1 element long maps would be unpacked and
> >>>>>>>       forwarded. 0
> >>>>>>>       > element
> >>>>>>>       > >>>>>>>>> maps
> >>>>>>>       > >>>>>>>>> would be published as delete. Any other count
> >>>>>>>       > >>>>>>>>> of map entries is in "waiting for correct
> deletes to
> >>>>>>>       > arrive"-state.
> >>>>>>>       > >>>>>>>>>
> >>>>>>>       > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>>       > >>>>>>>>>
> >>>>>>>       > >>>>>>>>> It does look like I could replace the second
> >>>>>>>       repartition store
> >>>>>>>       > and
> >>>>>>>       > >>>>>>>>>> highwater store with a groupBy and reduce.
> However,
> >>>>>>>       it looks
> >>>>>>>       > like
> >>>>>>>       > >>>>>>>>>> I
> >>>>>>>       > >>>>>>>>>> would
> >>>>>>>       > >>>>>>>>>> still need to store the highwater value within
> the
> >>>>>>>       materialized
> >>>>>>>       > >>>>>>>>>> store,
> >>>>>>>       > >>>>>>>>>>
> >>>>>>>       > >>>>>>>>>> to
> >>>>>>>       > >>>>>>>>> compare the arrival of out-of-order records
> (assuming
> >>>>>>>
> >>>>>> my
> >>>>
> >>>>>       > >>>>>>>>> understanding
> >>>>>>>       > >>>>>>>>> of
> >>>>>>>       > >>>>>>>>> THIS is correct...). This in effect is the same
> as
> >>>>>>> the
> >>>>>>>       design I
> >>>>>>>       > have
> >>>>>>>       > >>>>>>>>> now,
> >>>>>>>       > >>>>>>>>> just with the two tables merged together.
> >>>>>>>       > >>>>>>>>>
> >>>>>>>       >
> >>>>>>>       >
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>>
> >>>
> >>
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Will that work? I expected it to blow up with ClassCastException or similar.

You either would have to specify the window you fetch/put or iterate 
across all windows the key was found in right?

I just hope the window-store doesn't check stream-time under the hoods 
that would be a questionable interface.

If it does: did you see my comment on checking all the windows earlier?
that would be needed to actually give reasonable time gurantees.

Best



On 25.09.2018 13:18, Adam Bellemare wrote:
> Hi Jan
>
> Check for  " highwaterMat " in the PR. I only changed the state store, not
> the ProcessorSupplier.
>
> Thanks,
> Adam
>
> On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>>
>>
>> On 24.09.2018 16:26, Adam Bellemare wrote:
>>
>>> @Guozhang
>>>
>>> Thanks for the information. This is indeed something that will be
>>> extremely
>>> useful for this KIP.
>>>
>>> @Jan
>>> Thanks for your explanations. That being said, I will not be moving ahead
>>> with an implementation using reshuffle/groupBy solution as you propose.
>>> That being said, if you wish to implement it yourself off of my current PR
>>> and submit it as a competitive alternative, I would be more than happy to
>>> help vet that as an alternate solution. As it stands right now, I do not
>>> really have more time to invest into alternatives without there being a
>>> strong indication from the binding voters which they would prefer.
>>>
>>>
>> Hey, total no worries. I think I personally gave up on the streams DSL for
>> some time already, otherwise I would have pulled this KIP through already.
>> I am currently reimplementing my own DSL based on PAPI.
>>
>>
>>> I will look at finishing up my PR with the windowed state store in the
>>> next
>>> week or so, exercising it via tests, and then I will come back for final
>>> discussions. In the meantime, I hope that any of the binding voters could
>>> take a look at the KIP in the wiki. I have updated it according to the
>>> latest plan:
>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>> Support+non-key+joining+in+KTable
>>>
>>> I have also updated the KIP PR to use a windowed store. This could be
>>> replaced by the results of KIP-258 whenever they are completed.
>>> https://github.com/apache/kafka/pull/5527
>>>
>>> Thanks,
>>>
>>> Adam
>>>
>>
>> Is the HighWatermarkResolverProccessorsupplier already updated in the PR?
>> expected it to change to Windowed<K>,Long Missing something?
>>
>>
>>
>>>
>>>
>>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>
>>> Correction on my previous email: KAFKA-5533 is the wrong link, as it is
>>>> for
>>>> corresponding changelog mechanisms. But as part of KIP-258 we do want to
>>>> have "handling out-of-order data for source KTable" such that instead of
>>>> blindly apply the updates to the materialized store, i.e. following
>>>> offset
>>>> ordering, we will reject updates that are older than the current key's
>>>> timestamps, i.e. following timestamp ordering.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
>>>> wrote:
>>>>
>>>> Hello Adam,
>>>>>
>>>>> Thanks for the explanation. Regarding the final step (i.e. the high
>>>>> watermark store, now altered to be replaced with a window store), I
>>>>> think
>>>>> another current on-going KIP may actually help:
>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>>
>>>>>
>>>>> This is for adding the timestamp into a key-value store (i.e. only for
>>>>> non-windowed KTable), and then one of its usage, as described in
>>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
>>>>> "reject" updates from the source topics if its timestamp is smaller than
>>>>> the current key's latest update timestamp. I think it is very similar to
>>>>> what you have in mind for high watermark based filtering, while you only
>>>>> need to make sure that the timestamps of the joining records are
>>>>>
>>>> correctly
>>>>
>>>>> inherited though the whole topology to the final stage.
>>>>>
>>>>> Note that this KIP is for key-value store and hence non-windowed KTables
>>>>> only, but for windowed KTables we do not really have a good support for
>>>>> their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107)
>>>>> I
>>>>> think we can just consider non-windowed KTable-KTable non-key joins for
>>>>> now. In which case, KIP-258 should help.
>>>>>
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>>>>>>
>>>>> wrote:
>>>>>
>>>>>
>>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>>
>>>>>> Hi Guozhang
>>>>>>>
>>>>>>> Current highwater mark implementation would grow endlessly based on
>>>>>>> primary key of original event. It is a pair of (<this table primary
>>>>>>>
>>>>>> key>,
>>>>
>>>>> <highest offset seen for that key>). This is used to differentiate
>>>>>>>
>>>>>> between
>>>>
>>>>> late arrivals and new updates. My newest proposal would be to replace
>>>>>>>
>>>>>> it
>>>>
>>>>> with a Windowed state store of Duration N. This would allow the same
>>>>>>> behaviour, but cap the size based on time. This should allow for all
>>>>>>> late-arriving events to be processed, and should be customizable by
>>>>>>> the
>>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
>>>>>>>
>>>>>> window,
>>>>
>>>>> or perhaps 7 days...).
>>>>>>>
>>>>>>> Hi Adam, using time based retention can do the trick here. Even if I
>>>>>> would still like to see the automatic repartitioning optional since I
>>>>>>
>>>>> would
>>>>
>>>>> just reshuffle again. With windowed store I am a little bit sceptical
>>>>>>
>>>>> about
>>>>
>>>>> how to determine the window. So esentially one could run into problems
>>>>>>
>>>>> when
>>>>
>>>>> the rapid change happens near a window border. I will check you
>>>>>> implementation in detail, if its problematic, we could still check
>>>>>> _all_
>>>>>> windows on read with not to bad performance impact I guess. Will let
>>>>>> you
>>>>>> know if the implementation would be correct as is. I wouldn't not like
>>>>>>
>>>>> to
>>>>
>>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I
>>>>>>
>>>>> think
>>>>
>>>>> we can't expect that.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> @Jan
>>>>>>> I believe I understand what you mean now - thanks for the diagram, it
>>>>>>> did really help. You are correct that I do not have the original
>>>>>>>
>>>>>> primary
>>>>
>>>>> key available, and I can see that if it was available then you would be
>>>>>>> able to add and remove events from the Map. That being said, I
>>>>>>>
>>>>>> encourage
>>>>
>>>>> you to finish your diagrams / charts just for clarity for everyone
>>>>>>>
>>>>>> else.
>>>>
>>>>>
>>>>>>> Yeah 100%, this giphy thing is just really hard work. But I understand
>>>>>>>
>>>>>> the benefits for the rest. Sorry about the original primary key, We
>>>>>> have
>>>>>> join and Group by implemented our own in PAPI and basically not using
>>>>>>
>>>>> any
>>>>
>>>>> DSL (Just the abstraction). Completely missed that in original DSL its
>>>>>>
>>>>> not
>>>>
>>>>> there and just assumed it. total brain mess up on my end. Will finish
>>>>>>
>>>>> the
>>>>
>>>>> chart as soon as i get a quite evening this week.
>>>>>>
>>>>>> My follow up question for you is, won't the Map stay inside the State
>>>>>>
>>>>>>> Store indefinitely after all of the changes have propagated? Isn't
>>>>>>> this
>>>>>>> effectively the same as a highwater mark state store?
>>>>>>>
>>>>>>> Thing is that if the map is empty, substractor is gonna return `null`
>>>>>>
>>>>> and
>>>>
>>>>> the key is removed from the keyspace. But there is going to be a store
>>>>>> 100%, the good thing is that I can use this store directly for
>>>>>> materialize() / enableSendingOldValues() is a regular store, satisfying
>>>>>> all gurantees needed for further groupby / join. The Windowed store is
>>>>>>
>>>>> not
>>>>
>>>>> keeping the values, so for the next statefull operation we would
>>>>>> need to instantiate an extra store. or we have the window store also
>>>>>>
>>>>> have
>>>>
>>>>> the values then.
>>>>>>
>>>>>> Long story short. if we can flip in a custom group by before
>>>>>> repartitioning to the original primary key i think it would help the
>>>>>>
>>>>> users
>>>>
>>>>> big time in building efficient apps. Given the original primary key
>>>>>>
>>>>> issue I
>>>>
>>>>> understand that we do not have a solid foundation to build on.
>>>>>> Leaving primary key carry along to the user. very unfortunate. I could
>>>>>> understand the decision goes like that. I do not think its a good
>>>>>>
>>>>> decision.
>>>>
>>>>>
>>>>>>
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Thanks
>>>>>>> Adam
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
>>>>>>>
>>>>>> wrote:
>>>>
>>>>>
>>>>>>>       please remove me from this group
>>>>>>>
>>>>>>>       On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>>
>>>>>>>       wrote:
>>>>>>>
>>>>>>>       > Hi Adam,
>>>>>>>       >
>>>>>>>       > give me some time, will make such a chart. last time i didn't
>>>>>>>       get along
>>>>>>>       > well with giphy and ruined all your charts.
>>>>>>>       > Hopefully i can get it done today
>>>>>>>       >
>>>>>>>       > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>>       > > Hi Jan
>>>>>>>       > >
>>>>>>>       > > I have included a diagram of what I attempted on the KIP.
>>>>>>>       > >
>>>>>>>       >
>>>>>>>       https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>>       <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>>       > >
>>>>>>>       > > I attempted this back at the start of my own implementation
>>>>>>> of
>>>>>>>       this
>>>>>>>       > > solution, and since I could not get it to work I have since
>>>>>>>       discarded the
>>>>>>>       > > code. At this point in time, if you wish to continue pursuing
>>>>>>>       for your
>>>>>>>       > > groupBy solution, I ask that you please create a diagram on
>>>>>>>       the KIP
>>>>>>>       > > carefully explaining your solution. Please feel free to use
>>>>>>>       the image I
>>>>>>>       > > just posted as a starting point. I am having trouble
>>>>>>>       understanding your
>>>>>>>       > > explanations but I think that a carefully constructed diagram
>>>>>>>       will clear
>>>>>>>       > up
>>>>>>>       > > any misunderstandings. Alternately, please post a
>>>>>>>       comprehensive PR with
>>>>>>>       > > your solution. I can only guess at what you mean, and since I
>>>>>>>       value my
>>>>>>>       > own
>>>>>>>       > > time as much as you value yours, I believe it is your
>>>>>>>       responsibility to
>>>>>>>       > > provide an implementation instead of me trying to guess.
>>>>>>>       > >
>>>>>>>       > > Adam
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > >
>>>>>>>       > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>>
>>>>>>>       > > wrote:
>>>>>>>       > >
>>>>>>>       > >> Hi James,
>>>>>>>       > >>
>>>>>>>       > >> nice to see you beeing interested. kafka streams at this
>>>>>>>       point supports
>>>>>>>       > >> all sorts of joins as long as both streams have the same
>>>>>>> key.
>>>>>>>       > >> Adam is currently implementing a join where a KTable and a
>>>>>>>       KTable can
>>>>>>>       > have
>>>>>>>       > >> a one to many relation ship (1:n). We exploit that rocksdb
>>>>>>> is
>>>>>>>
>>>>>> a
>>>>
>>>>>       > >> datastore that keeps data sorted (At least exposes an API to
>>>>>>>       access the
>>>>>>>       > >> stored data in a sorted fashion).
>>>>>>>       > >>
>>>>>>>       > >> I think the technical caveats are well understood now and we
>>>>>>>
>>>>>> are
>>>>
>>>>>       > basically
>>>>>>>       > >> down to philosophy and API Design ( when Adam sees my newest
>>>>>>>       message).
>>>>>>>       > >> I have a lengthy track record of loosing those kinda
>>>>>>>       arguments within
>>>>>>>       > the
>>>>>>>       > >> streams community and I have no clue why. So I literally
>>>>>>>       can't wait for
>>>>>>>       > you
>>>>>>>       > >> to churn through this thread and give you opinion on how we
>>>>>>>       should
>>>>>>>       > design
>>>>>>>       > >> the return type of the oneToManyJoin and how many power we
>>>>>>>       want to give
>>>>>>>       > to
>>>>>>>       > >> the user vs "simplicity" (where simplicity isn't really that
>>>>>>>       as users
>>>>>>>       > still
>>>>>>>       > >> need to understand it I argue)
>>>>>>>       > >>
>>>>>>>       > >> waiting for you to join in on the discussion
>>>>>>>       > >>
>>>>>>>       > >> Best Jan
>>>>>>>       > >>
>>>>>>>       > >>
>>>>>>>       > >>
>>>>>>>       > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>>       > >>
>>>>>>>       > >>> I am new to this group and I found this subject
>>>>>>>       interesting.  Sounds
>>>>>>>       > like
>>>>>>>       > >>> you guys want to implement a join table of two streams? Is
>>>>>>> there
>>>>>>>       > somewhere
>>>>>>>       > >>> I can see the original requirement or proposal?
>>>>>>>       > >>>
>>>>>>>       > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>>       <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>>
>>>>>>>       > >>>> wrote:
>>>>>>>       > >>>>
>>>>>>>       > >>>>
>>>>>>>       > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>>>       > >>>>
>>>>>>>       > >>>>> I'm currently testing using a Windowed Store to store the
>>>>>>>       highwater
>>>>>>>       > >>>>> mark.
>>>>>>>       > >>>>> By all indications this should work fine, with the caveat
>>>>>>>       being that
>>>>>>>       > it
>>>>>>>       > >>>>> can
>>>>>>>       > >>>>> only resolve out-of-order arrival for up to the size of
>>>>>>>       the window
>>>>>>>       > (ie:
>>>>>>>       > >>>>> 24h, 72h, etc). This would remove the possibility of it
>>>>>>>
>>>>>> being
>>>>
>>>>>       > unbounded
>>>>>>>       > >>>>> in
>>>>>>>       > >>>>> size.
>>>>>>>       > >>>>>
>>>>>>>       > >>>>> With regards to Jan's suggestion, I believe this is where
>>>>>>>       we will
>>>>>>>       > have
>>>>>>>       > >>>>> to
>>>>>>>       > >>>>> remain in disagreement. While I do not disagree with your
>>>>>>>       statement
>>>>>>>       > >>>>> about
>>>>>>>       > >>>>> there likely to be additional joins done in a real-world
>>>>>>>       workflow, I
>>>>>>>       > do
>>>>>>>       > >>>>> not
>>>>>>>       > >>>>> see how you can conclusively deal with out-of-order
>>>>>>> arrival
>>>>>>> of
>>>>>>>       > >>>>> foreign-key
>>>>>>>       > >>>>> changes and subsequent joins. I have attempted what I
>>>>>>>       think you have
>>>>>>>       > >>>>> proposed (without a high-water, using groupBy and reduce)
>>>>>>>       and found
>>>>>>>       > >>>>> that if
>>>>>>>       > >>>>> the foreign key changes too quickly, or the load on a
>>>>>>>       stream thread
>>>>>>>       > is
>>>>>>>       > >>>>> too
>>>>>>>       > >>>>> high, the joined messages will arrive out-of-order and be
>>>>>>>       incorrectly
>>>>>>>       > >>>>> propagated, such that an intermediate event is
>>>>>>> represented
>>>>>>>       as the
>>>>>>>       > final
>>>>>>>       > >>>>> event.
>>>>>>>       > >>>>>
>>>>>>>       > >>>> Can you shed some light on your groupBy implementation.
>>>>>>>       There must be
>>>>>>>       > >>>> some sort of flaw in it.
>>>>>>>       > >>>> I have a suspicion where it is, I would just like to
>>>>>>>       confirm. The idea
>>>>>>>       > >>>> is bullet proof and it must be
>>>>>>>       > >>>> an implementation mess up. I would like to clarify before
>>>>>>>       we draw a
>>>>>>>       > >>>> conclusion.
>>>>>>>       > >>>>
>>>>>>>       > >>>>    Repartitioning the scattered events back to their
>>>>>>>
>>>>>> original
>>>>
>>>>>       > >>>>> partitions is the only way I know how to conclusively deal
>>>>>>>       with
>>>>>>>       > >>>>> out-of-order events in a given time frame, and to ensure
>>>>>>>       that the
>>>>>>>       > data
>>>>>>>       > >>>>> is
>>>>>>>       > >>>>> eventually consistent with the input events.
>>>>>>>       > >>>>>
>>>>>>>       > >>>>> If you have some code to share that illustrates your
>>>>>>>       approach, I
>>>>>>>       > would
>>>>>>>       > >>>>> be
>>>>>>>       > >>>>> very grateful as it would remove any misunderstandings
>>>>>>>       that I may
>>>>>>>       > have.
>>>>>>>       > >>>>>
>>>>>>>       > >>>> ah okay you were looking for my code. I don't have
>>>>>>>       something easily
>>>>>>>       > >>>> readable here as its bloated with OO-patterns.
>>>>>>>       > >>>>
>>>>>>>       > >>>> its anyhow trivial:
>>>>>>>       > >>>>
>>>>>>>       > >>>> @Override
>>>>>>>       > >>>>      public T apply(K aggKey, V value, T aggregate)
>>>>>>>       > >>>>      {
>>>>>>>       > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate);
>>>>>>> <<
>>>>>>>       imaginary
>>>>>>>       > >>>>          U toModifyKey = mapper.apply(value);
>>>>>>>       > >>>>              << this is the place where people actually
>>>>>>>       gonna have
>>>>>>>       > issues
>>>>>>>       > >>>> and why you probably couldn't do it. we would need to find
>>>>>>>       a solution
>>>>>>>       > here.
>>>>>>>       > >>>> I didn't realize that yet.
>>>>>>>       > >>>>              << we propagate the field in the joiner, so
>>>>>>>       that we can
>>>>>>>       > pick
>>>>>>>       > >>>> it up in an aggregate. Probably you have not thought of
>>>>>>>       this in your
>>>>>>>       > >>>> approach right?
>>>>>>>       > >>>>              << I am very open to find a generic solution
>>>>>>>       here. In my
>>>>>>>       > >>>> honest opinion this is broken in KTableImpl.GroupBy that
>>>>>>> it
>>>>>>>       looses
>>>>>>>       > the keys
>>>>>>>       > >>>> and only maintains the aggregate key.
>>>>>>>       > >>>>              << I abstracted it away back then way before
>>>>>>> i
>>>>>>> was
>>>>>>>       > thinking
>>>>>>>       > >>>> of oneToMany join. That is why I didn't realize its
>>>>>>>       significance here.
>>>>>>>       > >>>>              << Opinions?
>>>>>>>       > >>>>
>>>>>>>       > >>>>          for (V m : current)
>>>>>>>       > >>>>          {
>>>>>>>       > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>>>       > >>>>          }
>>>>>>>       > >>>>          if (isAdder)
>>>>>>>       > >>>>          {
>>>>>>>       > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>>>       > >>>>          }
>>>>>>>       > >>>>          else
>>>>>>>       > >>>>          {
>>>>>>>       > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>>       > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>>       > >>>>                  return null;
>>>>>>>       > >>>>              }
>>>>>>>       > >>>>          }
>>>>>>>       > >>>>          retrun asAggregateType(currentStateAsMap)
>>>>>>>       > >>>>      }
>>>>>>>       > >>>>
>>>>>>>       > >>>>
>>>>>>>       > >>>>
>>>>>>>       > >>>>
>>>>>>>       > >>>>
>>>>>>>       > >>>> Thanks,
>>>>>>>       > >>>>> Adam
>>>>>>>       > >>>>>
>>>>>>>       > >>>>>
>>>>>>>       > >>>>>
>>>>>>>       > >>>>>
>>>>>>>       > >>>>>
>>>>>>>       > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>>>>>       > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>>
>>>>>>>       > >>>>> wrote:
>>>>>>>       > >>>>>
>>>>>>>       > >>>>> Thanks Adam for bringing Matthias to speed!
>>>>>>>       > >>>>>> about the differences. I think re-keying back should be
>>>>>>>       optional at
>>>>>>>       > >>>>>> best.
>>>>>>>       > >>>>>> I would say we return a KScatteredTable with reshuffle()
>>>>>>>       returning
>>>>>>>       > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>>>>>       repartitioning
>>>>>>>       > >>>>>> optional.
>>>>>>>       > >>>>>> I am also in a big favour of doing the out of order
>>>>>>>       processing using
>>>>>>>       > >>>>>> group
>>>>>>>       > >>>>>> by instead high water mark tracking.
>>>>>>>       > >>>>>> Just because unbounded growth is just scary + It saves
>>>>>>> us
>>>>>>>       the header
>>>>>>>       > >>>>>> stuff.
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>> I think the abstraction of always repartitioning back is
>>>>>>>       just not so
>>>>>>>       > >>>>>> strong. Like the work has been done before we partition
>>>>>>>       back and
>>>>>>>       > >>>>>> grouping
>>>>>>>       > >>>>>> by something else afterwards is really common.
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>>>       > >>>>>>
>>>>>>>       > >>>>>> Hi Matthias
>>>>>>>       > >>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> While name spacing would be possible, it would require
>>>>>>> to
>>>>>>>       > deserialize
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>>>       suggest to
>>>>>>>       > no
>>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>>
>>>>>> becomes a
>>>>
>>>>>       > problem in
>>>>>>>       > >>>>>>>> the future, we can still add name spacing later on.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Agreed. I will go with using a reserved string and
>>>>>>>       document it.
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> My main concern about the design it the type of the
>>>>>>>       result KTable:
>>>>>>>       > If
>>>>>>>       > >>>>>>> I
>>>>>>>       > >>>>>>> understood the proposal correctly,
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> In your example, you have table1 and table2 swapped.
>>>>>>>       Here is how it
>>>>>>>       > >>>>>>> works
>>>>>>>       > >>>>>>> currently:
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> 1) table1 has the records that contain the foreign key
>>>>>>>       within their
>>>>>>>       > >>>>>>> value.
>>>>>>>       > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>>> <b,(fk=A,bar=2)>,
>>>>>>>       > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>>       > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> 2) A Value mapper is required to extract the foreign
>>>>>>> key.
>>>>>>>       > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>>>>>       <http://value.fk> )
>>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> The mapper is applied to each element in table1, and a
>>>>>>>       new combined
>>>>>>>       > >>>>>>> key is
>>>>>>>       > >>>>>>> made:
>>>>>>>       > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>> (fk=A,bar=2)>,
>>>>>>>       <B-c,
>>>>>>>       > >>>>>>> (fk=B,bar=3)>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>>       > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>>       (fk=A,bar=2)>
>>>>>>>       > >>>>>>> Table2: <A,X>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>>       > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>>       > >>>>>>> Table2: <B,Y>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> 4) From here, they can be joined together locally by
>>>>>>>       applying the
>>>>>>>       > >>>>>>> joiner
>>>>>>>       > >>>>>>> function.
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> At this point, Jan's design and my design deviate. My
>>>>>>>       design goes
>>>>>>>       > on
>>>>>>>       > >>>>>>> to
>>>>>>>       > >>>>>>> repartition the data post-join and resolve out-of-order
>>>>>>>       arrival of
>>>>>>>       > >>>>>>> records,
>>>>>>>       > >>>>>>> finally returning the data keyed just the original key.
>>>>>>>       I do not
>>>>>>>       > >>>>>>> expose
>>>>>>>       > >>>>>>> the
>>>>>>>       > >>>>>>> CombinedKey or any of the internals outside of the
>>>>>>>       joinOnForeignKey
>>>>>>>       > >>>>>>> function. This does make for larger footprint, but it
>>>>>>>       removes all
>>>>>>>       > >>>>>>> agency
>>>>>>>       > >>>>>>> for resolving out-of-order arrivals and handling
>>>>>>>       CombinedKeys from
>>>>>>>       > the
>>>>>>>       > >>>>>>> user. I believe that this makes the function much
>>>>>>> easier
>>>>>>>       to use.
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> Let me know if this helps resolve your questions, and
>>>>>>>       please feel
>>>>>>>       > >>>>>>> free to
>>>>>>>       > >>>>>>> add anything else on your mind.
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> Thanks again,
>>>>>>>       > >>>>>>> Adam
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>>>       > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>>>>>>>
>>>>>>>       > >>>>>>> wrote:
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>> Hi,
>>>>>>>       > >>>>>>>
>>>>>>>       > >>>>>>>> I am just catching up on this thread. I did not read
>>>>>>>       everything so
>>>>>>>       > >>>>>>>> far,
>>>>>>>       > >>>>>>>> but want to share couple of initial thoughts:
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Headers: I think there is a fundamental difference
>>>>>>>       between header
>>>>>>>       > >>>>>>>> usage
>>>>>>>       > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>>>>>>>       changelog topic
>>>>>>>       > >>>>>>>> that
>>>>>>>       > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>>>>>>>       to write
>>>>>>>       > into
>>>>>>>       > >>>>>>>> them. In fact, no user header are written into the
>>>>>>>       changelog topic
>>>>>>>       > >>>>>>>> and
>>>>>>>       > >>>>>>>> thus, there are not conflicts.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Nevertheless, I don't see a big issue with using
>>>>>>>       headers within
>>>>>>>       > >>>>>>>> Streams.
>>>>>>>       > >>>>>>>> As long as we document it, we can have some "reserved"
>>>>>>>       header keys
>>>>>>>       > >>>>>>>> and
>>>>>>>       > >>>>>>>> users are not allowed to use when processing data with
>>>>>>>       Kafka
>>>>>>>       > Streams.
>>>>>>>       > >>>>>>>> IMHO, this should be ok.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> I think there is a safe way to avoid conflicts, since
>>>>>>> these
>>>>>>>       > headers
>>>>>>>       > >>>>>>>> are
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>> only needed in internal topics (I think):
>>>>>>>       > >>>>>>>>> For internal and changelog topics, we can namespace
>>>>>>>       all headers:
>>>>>>>       > >>>>>>>>> * user-defined headers are namespaced as "external."
>>>>>>> +
>>>>>>>       headerKey
>>>>>>>       > >>>>>>>>> * internal headers are namespaced as "internal." +
>>>>>>>       headerKey
>>>>>>>       > >>>>>>>>>
>>>>>>>       > >>>>>>>>> While name spacing would be possible, it would
>>>>>>> require
>>>>>>>
>>>>>> to
>>>>
>>>>>       > >>>>>>>> deserialize
>>>>>>>       > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>>>       suggest to
>>>>>>>       > no
>>>>>>>       > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>>
>>>>>> becomes a
>>>>
>>>>>       > problem in
>>>>>>>       > >>>>>>>> the future, we can still add name spacing later on.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> My main concern about the design it the type of the
>>>>>>>       result KTable:
>>>>>>>       > >>>>>>>> If I
>>>>>>>       > >>>>>>>> understood the proposal correctly,
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>       > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> implies that the `joinedTable` has the same key as the
>>>>>>>       left input
>>>>>>>       > >>>>>>>> table.
>>>>>>>       > >>>>>>>> IMHO, this does not work because if table2 contains
>>>>>>>       multiple rows
>>>>>>>       > >>>>>>>> that
>>>>>>>       > >>>>>>>> join with a record in table1 (what is the main purpose
>>>>>>>
>>>>>> of
>>>>
>>>>> a
>>>>>>>       > foreign
>>>>>>>       > >>>>>>>> key
>>>>>>>       > >>>>>>>> join), the result table would only contain a single
>>>>>>>       join result,
>>>>>>>       > but
>>>>>>>       > >>>>>>>> not
>>>>>>>       > >>>>>>>> multiple.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Example:
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> table1 input stream: <A,X>
>>>>>>>       > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>>>>>>>       "A" joins).
>>>>>>>       > If
>>>>>>>       > >>>>>>>> the
>>>>>>>       > >>>>>>>> result key is the same key as key of table1, this
>>>>>>>       implies that the
>>>>>>>       > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>>>>>>>       but not
>>>>>>>       > both.
>>>>>>>       > >>>>>>>> Because the share the same key, whatever result record
>>>>>>>       we emit
>>>>>>>       > later,
>>>>>>>       > >>>>>>>> overwrite the previous result.
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> This is the reason why Jan originally proposed to use
>>>>>>> a
>>>>>>>       > combination
>>>>>>>       > >>>>>>>> of
>>>>>>>       > >>>>>>>> both primary keys of the input tables as key of the
>>>>>>>       output table.
>>>>>>>       > >>>>>>>> This
>>>>>>>       > >>>>>>>> makes the keys of the output table unique and we can
>>>>>>>       store both in
>>>>>>>       > >>>>>>>> the
>>>>>>>       > >>>>>>>> output table:
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Thoughts?
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> -Matthias
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>       > >>>>>>>>
>>>>>>>       > >>>>>>>> Just on remark here.
>>>>>>>       > >>>>>>>>> The high-watermark could be disregarded. The decision
>>>>>>>       about the
>>>>>>>       > >>>>>>>>> forward
>>>>>>>       > >>>>>>>>> depends on the size of the aggregated map.
>>>>>>>       > >>>>>>>>> Only 1 element long maps would be unpacked and
>>>>>>>       forwarded. 0
>>>>>>>       > element
>>>>>>>       > >>>>>>>>> maps
>>>>>>>       > >>>>>>>>> would be published as delete. Any other count
>>>>>>>       > >>>>>>>>> of map entries is in "waiting for correct deletes to
>>>>>>>       > arrive"-state.
>>>>>>>       > >>>>>>>>>
>>>>>>>       > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>       > >>>>>>>>>
>>>>>>>       > >>>>>>>>> It does look like I could replace the second
>>>>>>>       repartition store
>>>>>>>       > and
>>>>>>>       > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>>>>>>>       it looks
>>>>>>>       > like
>>>>>>>       > >>>>>>>>>> I
>>>>>>>       > >>>>>>>>>> would
>>>>>>>       > >>>>>>>>>> still need to store the highwater value within the
>>>>>>>       materialized
>>>>>>>       > >>>>>>>>>> store,
>>>>>>>       > >>>>>>>>>>
>>>>>>>       > >>>>>>>>>> to
>>>>>>>       > >>>>>>>>> compare the arrival of out-of-order records (assuming
>>>>>>>
>>>>>> my
>>>>
>>>>>       > >>>>>>>>> understanding
>>>>>>>       > >>>>>>>>> of
>>>>>>>       > >>>>>>>>> THIS is correct...). This in effect is the same as
>>>>>>> the
>>>>>>>       design I
>>>>>>>       > have
>>>>>>>       > >>>>>>>>> now,
>>>>>>>       > >>>>>>>>> just with the two tables merged together.
>>>>>>>       > >>>>>>>>>
>>>>>>>       >
>>>>>>>       >
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> -- Guozhang
>>>>
>>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Jan

Check for  " highwaterMat " in the PR. I only changed the state store, not
the ProcessorSupplier.

Thanks,
Adam

On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

>
>
> On 24.09.2018 16:26, Adam Bellemare wrote:
>
>> @Guozhang
>>
>> Thanks for the information. This is indeed something that will be
>> extremely
>> useful for this KIP.
>>
>> @Jan
>> Thanks for your explanations. That being said, I will not be moving ahead
>> with an implementation using reshuffle/groupBy solution as you propose.
>> That being said, if you wish to implement it yourself off of my current PR
>> and submit it as a competitive alternative, I would be more than happy to
>> help vet that as an alternate solution. As it stands right now, I do not
>> really have more time to invest into alternatives without there being a
>> strong indication from the binding voters which they would prefer.
>>
>>
> Hey, total no worries. I think I personally gave up on the streams DSL for
> some time already, otherwise I would have pulled this KIP through already.
> I am currently reimplementing my own DSL based on PAPI.
>
>
>> I will look at finishing up my PR with the windowed state store in the
>> next
>> week or so, exercising it via tests, and then I will come back for final
>> discussions. In the meantime, I hope that any of the binding voters could
>> take a look at the KIP in the wiki. I have updated it according to the
>> latest plan:
>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> Support+non-key+joining+in+KTable
>>
>> I have also updated the KIP PR to use a windowed store. This could be
>> replaced by the results of KIP-258 whenever they are completed.
>> https://github.com/apache/kafka/pull/5527
>>
>> Thanks,
>>
>> Adam
>>
>
> Is the HighWatermarkResolverProccessorsupplier already updated in the PR?
> expected it to change to Windowed<K>,Long Missing something?
>
>
>
>>
>>
>> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>
>> Correction on my previous email: KAFKA-5533 is the wrong link, as it is
>>> for
>>> corresponding changelog mechanisms. But as part of KIP-258 we do want to
>>> have "handling out-of-order data for source KTable" such that instead of
>>> blindly apply the updates to the materialized store, i.e. following
>>> offset
>>> ordering, we will reject updates that are older than the current key's
>>> timestamps, i.e. following timestamp ordering.
>>>
>>>
>>> Guozhang
>>>
>>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>
>>> Hello Adam,
>>>>
>>>> Thanks for the explanation. Regarding the final step (i.e. the high
>>>> watermark store, now altered to be replaced with a window store), I
>>>> think
>>>> another current on-going KIP may actually help:
>>>>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>>
>>>>
>>>> This is for adding the timestamp into a key-value store (i.e. only for
>>>> non-windowed KTable), and then one of its usage, as described in
>>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
>>>> "reject" updates from the source topics if its timestamp is smaller than
>>>> the current key's latest update timestamp. I think it is very similar to
>>>> what you have in mind for high watermark based filtering, while you only
>>>> need to make sure that the timestamps of the joining records are
>>>>
>>> correctly
>>>
>>>> inherited though the whole topology to the final stage.
>>>>
>>>> Note that this KIP is for key-value store and hence non-windowed KTables
>>>> only, but for windowed KTables we do not really have a good support for
>>>> their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107)
>>>> I
>>>> think we can just consider non-windowed KTable-KTable non-key joins for
>>>> now. In which case, KIP-258 should help.
>>>>
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>>>> >
>>>> wrote:
>>>>
>>>>
>>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>>
>>>>> Hi Guozhang
>>>>>>
>>>>>> Current highwater mark implementation would grow endlessly based on
>>>>>> primary key of original event. It is a pair of (<this table primary
>>>>>>
>>>>> key>,
>>>
>>>> <highest offset seen for that key>). This is used to differentiate
>>>>>>
>>>>> between
>>>
>>>> late arrivals and new updates. My newest proposal would be to replace
>>>>>>
>>>>> it
>>>
>>>> with a Windowed state store of Duration N. This would allow the same
>>>>>> behaviour, but cap the size based on time. This should allow for all
>>>>>> late-arriving events to be processed, and should be customizable by
>>>>>> the
>>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
>>>>>>
>>>>> window,
>>>
>>>> or perhaps 7 days...).
>>>>>>
>>>>>> Hi Adam, using time based retention can do the trick here. Even if I
>>>>> would still like to see the automatic repartitioning optional since I
>>>>>
>>>> would
>>>
>>>> just reshuffle again. With windowed store I am a little bit sceptical
>>>>>
>>>> about
>>>
>>>> how to determine the window. So esentially one could run into problems
>>>>>
>>>> when
>>>
>>>> the rapid change happens near a window border. I will check you
>>>>> implementation in detail, if its problematic, we could still check
>>>>> _all_
>>>>> windows on read with not to bad performance impact I guess. Will let
>>>>> you
>>>>> know if the implementation would be correct as is. I wouldn't not like
>>>>>
>>>> to
>>>
>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I
>>>>>
>>>> think
>>>
>>>> we can't expect that.
>>>>>
>>>>>
>>>>>>
>>>>>> @Jan
>>>>>> I believe I understand what you mean now - thanks for the diagram, it
>>>>>> did really help. You are correct that I do not have the original
>>>>>>
>>>>> primary
>>>
>>>> key available, and I can see that if it was available then you would be
>>>>>> able to add and remove events from the Map. That being said, I
>>>>>>
>>>>> encourage
>>>
>>>> you to finish your diagrams / charts just for clarity for everyone
>>>>>>
>>>>> else.
>>>
>>>>
>>>>>> Yeah 100%, this giphy thing is just really hard work. But I understand
>>>>>>
>>>>> the benefits for the rest. Sorry about the original primary key, We
>>>>> have
>>>>> join and Group by implemented our own in PAPI and basically not using
>>>>>
>>>> any
>>>
>>>> DSL (Just the abstraction). Completely missed that in original DSL its
>>>>>
>>>> not
>>>
>>>> there and just assumed it. total brain mess up on my end. Will finish
>>>>>
>>>> the
>>>
>>>> chart as soon as i get a quite evening this week.
>>>>>
>>>>> My follow up question for you is, won't the Map stay inside the State
>>>>>
>>>>>> Store indefinitely after all of the changes have propagated? Isn't
>>>>>> this
>>>>>> effectively the same as a highwater mark state store?
>>>>>>
>>>>>> Thing is that if the map is empty, substractor is gonna return `null`
>>>>>
>>>> and
>>>
>>>> the key is removed from the keyspace. But there is going to be a store
>>>>> 100%, the good thing is that I can use this store directly for
>>>>> materialize() / enableSendingOldValues() is a regular store, satisfying
>>>>> all gurantees needed for further groupby / join. The Windowed store is
>>>>>
>>>> not
>>>
>>>> keeping the values, so for the next statefull operation we would
>>>>> need to instantiate an extra store. or we have the window store also
>>>>>
>>>> have
>>>
>>>> the values then.
>>>>>
>>>>> Long story short. if we can flip in a custom group by before
>>>>> repartitioning to the original primary key i think it would help the
>>>>>
>>>> users
>>>
>>>> big time in building efficient apps. Given the original primary key
>>>>>
>>>> issue I
>>>
>>>> understand that we do not have a solid foundation to build on.
>>>>> Leaving primary key carry along to the user. very unfortunate. I could
>>>>> understand the decision goes like that. I do not think its a good
>>>>>
>>>> decision.
>>>
>>>>
>>>>>
>>>>>
>>>>>>
>>>>>>
>>>>>> Thanks
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
>>>>>>
>>>>> wrote:
>>>
>>>>
>>>>>>      please remove me from this group
>>>>>>
>>>>>>      On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>
>>>>>>      wrote:
>>>>>>
>>>>>>      > Hi Adam,
>>>>>>      >
>>>>>>      > give me some time, will make such a chart. last time i didn't
>>>>>>      get along
>>>>>>      > well with giphy and ruined all your charts.
>>>>>>      > Hopefully i can get it done today
>>>>>>      >
>>>>>>      > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>>      > > Hi Jan
>>>>>>      > >
>>>>>>      > > I have included a diagram of what I attempted on the KIP.
>>>>>>      > >
>>>>>>      >
>>>>>>      https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>>      <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>>      > >
>>>>>>      > > I attempted this back at the start of my own implementation
>>>>>> of
>>>>>>      this
>>>>>>      > > solution, and since I could not get it to work I have since
>>>>>>      discarded the
>>>>>>      > > code. At this point in time, if you wish to continue pursuing
>>>>>>      for your
>>>>>>      > > groupBy solution, I ask that you please create a diagram on
>>>>>>      the KIP
>>>>>>      > > carefully explaining your solution. Please feel free to use
>>>>>>      the image I
>>>>>>      > > just posted as a starting point. I am having trouble
>>>>>>      understanding your
>>>>>>      > > explanations but I think that a carefully constructed diagram
>>>>>>      will clear
>>>>>>      > up
>>>>>>      > > any misunderstandings. Alternately, please post a
>>>>>>      comprehensive PR with
>>>>>>      > > your solution. I can only guess at what you mean, and since I
>>>>>>      value my
>>>>>>      > own
>>>>>>      > > time as much as you value yours, I believe it is your
>>>>>>      responsibility to
>>>>>>      > > provide an implementation instead of me trying to guess.
>>>>>>      > >
>>>>>>      > > Adam
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > >
>>>>>>      > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>
>>>>>>      > > wrote:
>>>>>>      > >
>>>>>>      > >> Hi James,
>>>>>>      > >>
>>>>>>      > >> nice to see you beeing interested. kafka streams at this
>>>>>>      point supports
>>>>>>      > >> all sorts of joins as long as both streams have the same
>>>>>> key.
>>>>>>      > >> Adam is currently implementing a join where a KTable and a
>>>>>>      KTable can
>>>>>>      > have
>>>>>>      > >> a one to many relation ship (1:n). We exploit that rocksdb
>>>>>> is
>>>>>>
>>>>> a
>>>
>>>>      > >> datastore that keeps data sorted (At least exposes an API to
>>>>>>      access the
>>>>>>      > >> stored data in a sorted fashion).
>>>>>>      > >>
>>>>>>      > >> I think the technical caveats are well understood now and we
>>>>>>
>>>>> are
>>>
>>>>      > basically
>>>>>>      > >> down to philosophy and API Design ( when Adam sees my newest
>>>>>>      message).
>>>>>>      > >> I have a lengthy track record of loosing those kinda
>>>>>>      arguments within
>>>>>>      > the
>>>>>>      > >> streams community and I have no clue why. So I literally
>>>>>>      can't wait for
>>>>>>      > you
>>>>>>      > >> to churn through this thread and give you opinion on how we
>>>>>>      should
>>>>>>      > design
>>>>>>      > >> the return type of the oneToManyJoin and how many power we
>>>>>>      want to give
>>>>>>      > to
>>>>>>      > >> the user vs "simplicity" (where simplicity isn't really that
>>>>>>      as users
>>>>>>      > still
>>>>>>      > >> need to understand it I argue)
>>>>>>      > >>
>>>>>>      > >> waiting for you to join in on the discussion
>>>>>>      > >>
>>>>>>      > >> Best Jan
>>>>>>      > >>
>>>>>>      > >>
>>>>>>      > >>
>>>>>>      > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>>      > >>
>>>>>>      > >>> I am new to this group and I found this subject
>>>>>>      interesting.  Sounds
>>>>>>      > like
>>>>>>      > >>> you guys want to implement a join table of two streams? Is
>>>>>> there
>>>>>>      > somewhere
>>>>>>      > >>> I can see the original requirement or proposal?
>>>>>>      > >>>
>>>>>>      > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>
>>>>>>      > >>>> wrote:
>>>>>>      > >>>>
>>>>>>      > >>>>
>>>>>>      > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>>      > >>>>
>>>>>>      > >>>>> I'm currently testing using a Windowed Store to store the
>>>>>>      highwater
>>>>>>      > >>>>> mark.
>>>>>>      > >>>>> By all indications this should work fine, with the caveat
>>>>>>      being that
>>>>>>      > it
>>>>>>      > >>>>> can
>>>>>>      > >>>>> only resolve out-of-order arrival for up to the size of
>>>>>>      the window
>>>>>>      > (ie:
>>>>>>      > >>>>> 24h, 72h, etc). This would remove the possibility of it
>>>>>>
>>>>> being
>>>
>>>>      > unbounded
>>>>>>      > >>>>> in
>>>>>>      > >>>>> size.
>>>>>>      > >>>>>
>>>>>>      > >>>>> With regards to Jan's suggestion, I believe this is where
>>>>>>      we will
>>>>>>      > have
>>>>>>      > >>>>> to
>>>>>>      > >>>>> remain in disagreement. While I do not disagree with your
>>>>>>      statement
>>>>>>      > >>>>> about
>>>>>>      > >>>>> there likely to be additional joins done in a real-world
>>>>>>      workflow, I
>>>>>>      > do
>>>>>>      > >>>>> not
>>>>>>      > >>>>> see how you can conclusively deal with out-of-order
>>>>>> arrival
>>>>>> of
>>>>>>      > >>>>> foreign-key
>>>>>>      > >>>>> changes and subsequent joins. I have attempted what I
>>>>>>      think you have
>>>>>>      > >>>>> proposed (without a high-water, using groupBy and reduce)
>>>>>>      and found
>>>>>>      > >>>>> that if
>>>>>>      > >>>>> the foreign key changes too quickly, or the load on a
>>>>>>      stream thread
>>>>>>      > is
>>>>>>      > >>>>> too
>>>>>>      > >>>>> high, the joined messages will arrive out-of-order and be
>>>>>>      incorrectly
>>>>>>      > >>>>> propagated, such that an intermediate event is
>>>>>> represented
>>>>>>      as the
>>>>>>      > final
>>>>>>      > >>>>> event.
>>>>>>      > >>>>>
>>>>>>      > >>>> Can you shed some light on your groupBy implementation.
>>>>>>      There must be
>>>>>>      > >>>> some sort of flaw in it.
>>>>>>      > >>>> I have a suspicion where it is, I would just like to
>>>>>>      confirm. The idea
>>>>>>      > >>>> is bullet proof and it must be
>>>>>>      > >>>> an implementation mess up. I would like to clarify before
>>>>>>      we draw a
>>>>>>      > >>>> conclusion.
>>>>>>      > >>>>
>>>>>>      > >>>>    Repartitioning the scattered events back to their
>>>>>>
>>>>> original
>>>
>>>>      > >>>>> partitions is the only way I know how to conclusively deal
>>>>>>      with
>>>>>>      > >>>>> out-of-order events in a given time frame, and to ensure
>>>>>>      that the
>>>>>>      > data
>>>>>>      > >>>>> is
>>>>>>      > >>>>> eventually consistent with the input events.
>>>>>>      > >>>>>
>>>>>>      > >>>>> If you have some code to share that illustrates your
>>>>>>      approach, I
>>>>>>      > would
>>>>>>      > >>>>> be
>>>>>>      > >>>>> very grateful as it would remove any misunderstandings
>>>>>>      that I may
>>>>>>      > have.
>>>>>>      > >>>>>
>>>>>>      > >>>> ah okay you were looking for my code. I don't have
>>>>>>      something easily
>>>>>>      > >>>> readable here as its bloated with OO-patterns.
>>>>>>      > >>>>
>>>>>>      > >>>> its anyhow trivial:
>>>>>>      > >>>>
>>>>>>      > >>>> @Override
>>>>>>      > >>>>      public T apply(K aggKey, V value, T aggregate)
>>>>>>      > >>>>      {
>>>>>>      > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate);
>>>>>> <<
>>>>>>      imaginary
>>>>>>      > >>>>          U toModifyKey = mapper.apply(value);
>>>>>>      > >>>>              << this is the place where people actually
>>>>>>      gonna have
>>>>>>      > issues
>>>>>>      > >>>> and why you probably couldn't do it. we would need to find
>>>>>>      a solution
>>>>>>      > here.
>>>>>>      > >>>> I didn't realize that yet.
>>>>>>      > >>>>              << we propagate the field in the joiner, so
>>>>>>      that we can
>>>>>>      > pick
>>>>>>      > >>>> it up in an aggregate. Probably you have not thought of
>>>>>>      this in your
>>>>>>      > >>>> approach right?
>>>>>>      > >>>>              << I am very open to find a generic solution
>>>>>>      here. In my
>>>>>>      > >>>> honest opinion this is broken in KTableImpl.GroupBy that
>>>>>> it
>>>>>>      looses
>>>>>>      > the keys
>>>>>>      > >>>> and only maintains the aggregate key.
>>>>>>      > >>>>              << I abstracted it away back then way before
>>>>>> i
>>>>>> was
>>>>>>      > thinking
>>>>>>      > >>>> of oneToMany join. That is why I didn't realize its
>>>>>>      significance here.
>>>>>>      > >>>>              << Opinions?
>>>>>>      > >>>>
>>>>>>      > >>>>          for (V m : current)
>>>>>>      > >>>>          {
>>>>>>      > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>>      > >>>>          }
>>>>>>      > >>>>          if (isAdder)
>>>>>>      > >>>>          {
>>>>>>      > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>>      > >>>>          }
>>>>>>      > >>>>          else
>>>>>>      > >>>>          {
>>>>>>      > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>>      > >>>> if(currentStateAsMap.isEmpty()){
>>>>>>      > >>>>                  return null;
>>>>>>      > >>>>              }
>>>>>>      > >>>>          }
>>>>>>      > >>>>          retrun asAggregateType(currentStateAsMap)
>>>>>>      > >>>>      }
>>>>>>      > >>>>
>>>>>>      > >>>>
>>>>>>      > >>>>
>>>>>>      > >>>>
>>>>>>      > >>>>
>>>>>>      > >>>> Thanks,
>>>>>>      > >>>>> Adam
>>>>>>      > >>>>>
>>>>>>      > >>>>>
>>>>>>      > >>>>>
>>>>>>      > >>>>>
>>>>>>      > >>>>>
>>>>>>      > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>>>>      > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>>
>>>>>>      > >>>>> wrote:
>>>>>>      > >>>>>
>>>>>>      > >>>>> Thanks Adam for bringing Matthias to speed!
>>>>>>      > >>>>>> about the differences. I think re-keying back should be
>>>>>>      optional at
>>>>>>      > >>>>>> best.
>>>>>>      > >>>>>> I would say we return a KScatteredTable with reshuffle()
>>>>>>      returning
>>>>>>      > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>>>>      repartitioning
>>>>>>      > >>>>>> optional.
>>>>>>      > >>>>>> I am also in a big favour of doing the out of order
>>>>>>      processing using
>>>>>>      > >>>>>> group
>>>>>>      > >>>>>> by instead high water mark tracking.
>>>>>>      > >>>>>> Just because unbounded growth is just scary + It saves
>>>>>> us
>>>>>>      the header
>>>>>>      > >>>>>> stuff.
>>>>>>      > >>>>>>
>>>>>>      > >>>>>> I think the abstraction of always repartitioning back is
>>>>>>      just not so
>>>>>>      > >>>>>> strong. Like the work has been done before we partition
>>>>>>      back and
>>>>>>      > >>>>>> grouping
>>>>>>      > >>>>>> by something else afterwards is really common.
>>>>>>      > >>>>>>
>>>>>>      > >>>>>>
>>>>>>      > >>>>>>
>>>>>>      > >>>>>>
>>>>>>      > >>>>>>
>>>>>>      > >>>>>>
>>>>>>      > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>>      > >>>>>>
>>>>>>      > >>>>>> Hi Matthias
>>>>>>      > >>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> While name spacing would be possible, it would require
>>>>>> to
>>>>>>      > deserialize
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>>      suggest to
>>>>>>      > no
>>>>>>      > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>
>>>>> becomes a
>>>
>>>>      > problem in
>>>>>>      > >>>>>>>> the future, we can still add name spacing later on.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Agreed. I will go with using a reserved string and
>>>>>>      document it.
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> My main concern about the design it the type of the
>>>>>>      result KTable:
>>>>>>      > If
>>>>>>      > >>>>>>> I
>>>>>>      > >>>>>>> understood the proposal correctly,
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> In your example, you have table1 and table2 swapped.
>>>>>>      Here is how it
>>>>>>      > >>>>>>> works
>>>>>>      > >>>>>>> currently:
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> 1) table1 has the records that contain the foreign key
>>>>>>      within their
>>>>>>      > >>>>>>> value.
>>>>>>      > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>,
>>>>>> <b,(fk=A,bar=2)>,
>>>>>>      > >>>>>>> <c,(fk=B,bar=3)>
>>>>>>      > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> 2) A Value mapper is required to extract the foreign
>>>>>> key.
>>>>>>      > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>>>>      <http://value.fk> )
>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> The mapper is applied to each element in table1, and a
>>>>>>      new combined
>>>>>>      > >>>>>>> key is
>>>>>>      > >>>>>>> made:
>>>>>>      > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>> (fk=A,bar=2)>,
>>>>>>      <B-c,
>>>>>>      > >>>>>>> (fk=B,bar=3)>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> a) Stream Thread with Partition 0:
>>>>>>      > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>>      (fk=A,bar=2)>
>>>>>>      > >>>>>>> Table2: <A,X>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> b) Stream Thread with Partition 1:
>>>>>>      > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>      > >>>>>>> Table2: <B,Y>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> 4) From here, they can be joined together locally by
>>>>>>      applying the
>>>>>>      > >>>>>>> joiner
>>>>>>      > >>>>>>> function.
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> At this point, Jan's design and my design deviate. My
>>>>>>      design goes
>>>>>>      > on
>>>>>>      > >>>>>>> to
>>>>>>      > >>>>>>> repartition the data post-join and resolve out-of-order
>>>>>>      arrival of
>>>>>>      > >>>>>>> records,
>>>>>>      > >>>>>>> finally returning the data keyed just the original key.
>>>>>>      I do not
>>>>>>      > >>>>>>> expose
>>>>>>      > >>>>>>> the
>>>>>>      > >>>>>>> CombinedKey or any of the internals outside of the
>>>>>>      joinOnForeignKey
>>>>>>      > >>>>>>> function. This does make for larger footprint, but it
>>>>>>      removes all
>>>>>>      > >>>>>>> agency
>>>>>>      > >>>>>>> for resolving out-of-order arrivals and handling
>>>>>>      CombinedKeys from
>>>>>>      > the
>>>>>>      > >>>>>>> user. I believe that this makes the function much
>>>>>> easier
>>>>>>      to use.
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> Let me know if this helps resolve your questions, and
>>>>>>      please feel
>>>>>>      > >>>>>>> free to
>>>>>>      > >>>>>>> add anything else on your mind.
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> Thanks again,
>>>>>>      > >>>>>>> Adam
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>>      > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>>>>>>
>>>>>>      > >>>>>>> wrote:
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>> Hi,
>>>>>>      > >>>>>>>
>>>>>>      > >>>>>>>> I am just catching up on this thread. I did not read
>>>>>>      everything so
>>>>>>      > >>>>>>>> far,
>>>>>>      > >>>>>>>> but want to share couple of initial thoughts:
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Headers: I think there is a fundamental difference
>>>>>>      between header
>>>>>>      > >>>>>>>> usage
>>>>>>      > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>>>>>>      changelog topic
>>>>>>      > >>>>>>>> that
>>>>>>      > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>>>>>>      to write
>>>>>>      > into
>>>>>>      > >>>>>>>> them. In fact, no user header are written into the
>>>>>>      changelog topic
>>>>>>      > >>>>>>>> and
>>>>>>      > >>>>>>>> thus, there are not conflicts.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Nevertheless, I don't see a big issue with using
>>>>>>      headers within
>>>>>>      > >>>>>>>> Streams.
>>>>>>      > >>>>>>>> As long as we document it, we can have some "reserved"
>>>>>>      header keys
>>>>>>      > >>>>>>>> and
>>>>>>      > >>>>>>>> users are not allowed to use when processing data with
>>>>>>      Kafka
>>>>>>      > Streams.
>>>>>>      > >>>>>>>> IMHO, this should be ok.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> I think there is a safe way to avoid conflicts, since
>>>>>> these
>>>>>>      > headers
>>>>>>      > >>>>>>>> are
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>> only needed in internal topics (I think):
>>>>>>      > >>>>>>>>> For internal and changelog topics, we can namespace
>>>>>>      all headers:
>>>>>>      > >>>>>>>>> * user-defined headers are namespaced as "external."
>>>>>> +
>>>>>>      headerKey
>>>>>>      > >>>>>>>>> * internal headers are namespaced as "internal." +
>>>>>>      headerKey
>>>>>>      > >>>>>>>>>
>>>>>>      > >>>>>>>>> While name spacing would be possible, it would
>>>>>> require
>>>>>>
>>>>> to
>>>
>>>>      > >>>>>>>> deserialize
>>>>>>      > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>>      suggest to
>>>>>>      > no
>>>>>>      > >>>>>>>> namespace for now to avoid the overhead. If this
>>>>>>
>>>>> becomes a
>>>
>>>>      > problem in
>>>>>>      > >>>>>>>> the future, we can still add name spacing later on.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> My main concern about the design it the type of the
>>>>>>      result KTable:
>>>>>>      > >>>>>>>> If I
>>>>>>      > >>>>>>>> understood the proposal correctly,
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>      > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> implies that the `joinedTable` has the same key as the
>>>>>>      left input
>>>>>>      > >>>>>>>> table.
>>>>>>      > >>>>>>>> IMHO, this does not work because if table2 contains
>>>>>>      multiple rows
>>>>>>      > >>>>>>>> that
>>>>>>      > >>>>>>>> join with a record in table1 (what is the main purpose
>>>>>>
>>>>> of
>>>
>>>> a
>>>>>>      > foreign
>>>>>>      > >>>>>>>> key
>>>>>>      > >>>>>>>> join), the result table would only contain a single
>>>>>>      join result,
>>>>>>      > but
>>>>>>      > >>>>>>>> not
>>>>>>      > >>>>>>>> multiple.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Example:
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> table1 input stream: <A,X>
>>>>>>      > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>>>>>>      "A" joins).
>>>>>>      > If
>>>>>>      > >>>>>>>> the
>>>>>>      > >>>>>>>> result key is the same key as key of table1, this
>>>>>>      implies that the
>>>>>>      > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>>>>>>      but not
>>>>>>      > both.
>>>>>>      > >>>>>>>> Because the share the same key, whatever result record
>>>>>>      we emit
>>>>>>      > later,
>>>>>>      > >>>>>>>> overwrite the previous result.
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> This is the reason why Jan originally proposed to use
>>>>>> a
>>>>>>      > combination
>>>>>>      > >>>>>>>> of
>>>>>>      > >>>>>>>> both primary keys of the input tables as key of the
>>>>>>      output table.
>>>>>>      > >>>>>>>> This
>>>>>>      > >>>>>>>> makes the keys of the output table unique and we can
>>>>>>      store both in
>>>>>>      > >>>>>>>> the
>>>>>>      > >>>>>>>> output table:
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Thoughts?
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> -Matthias
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>      > >>>>>>>>
>>>>>>      > >>>>>>>> Just on remark here.
>>>>>>      > >>>>>>>>> The high-watermark could be disregarded. The decision
>>>>>>      about the
>>>>>>      > >>>>>>>>> forward
>>>>>>      > >>>>>>>>> depends on the size of the aggregated map.
>>>>>>      > >>>>>>>>> Only 1 element long maps would be unpacked and
>>>>>>      forwarded. 0
>>>>>>      > element
>>>>>>      > >>>>>>>>> maps
>>>>>>      > >>>>>>>>> would be published as delete. Any other count
>>>>>>      > >>>>>>>>> of map entries is in "waiting for correct deletes to
>>>>>>      > arrive"-state.
>>>>>>      > >>>>>>>>>
>>>>>>      > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>      > >>>>>>>>>
>>>>>>      > >>>>>>>>> It does look like I could replace the second
>>>>>>      repartition store
>>>>>>      > and
>>>>>>      > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>>>>>>      it looks
>>>>>>      > like
>>>>>>      > >>>>>>>>>> I
>>>>>>      > >>>>>>>>>> would
>>>>>>      > >>>>>>>>>> still need to store the highwater value within the
>>>>>>      materialized
>>>>>>      > >>>>>>>>>> store,
>>>>>>      > >>>>>>>>>>
>>>>>>      > >>>>>>>>>> to
>>>>>>      > >>>>>>>>> compare the arrival of out-of-order records (assuming
>>>>>>
>>>>> my
>>>
>>>>      > >>>>>>>>> understanding
>>>>>>      > >>>>>>>>> of
>>>>>>      > >>>>>>>>> THIS is correct...). This in effect is the same as
>>>>>> the
>>>>>>      design I
>>>>>>      > have
>>>>>>      > >>>>>>>>> now,
>>>>>>      > >>>>>>>>> just with the two tables merged together.
>>>>>>      > >>>>>>>>>
>>>>>>      >
>>>>>>      >
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>> --
>>>> -- Guozhang
>>>>
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

On 24.09.2018 16:26, Adam Bellemare wrote:
> @Guozhang
>
> Thanks for the information. This is indeed something that will be extremely
> useful for this KIP.
>
> @Jan
> Thanks for your explanations. That being said, I will not be moving ahead
> with an implementation using reshuffle/groupBy solution as you propose.
> That being said, if you wish to implement it yourself off of my current PR
> and submit it as a competitive alternative, I would be more than happy to
> help vet that as an alternate solution. As it stands right now, I do not
> really have more time to invest into alternatives without there being a
> strong indication from the binding voters which they would prefer.
>

Hey, total no worries. I think I personally gave up on the streams DSL 
for some time already, otherwise I would have pulled this KIP through 
already. I am currently reimplementing my own DSL based on PAPI.

>
> I will look at finishing up my PR with the windowed state store in the next
> week or so, exercising it via tests, and then I will come back for final
> discussions. In the meantime, I hope that any of the binding voters could
> take a look at the KIP in the wiki. I have updated it according to the
> latest plan:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable
>
> I have also updated the KIP PR to use a windowed store. This could be
> replaced by the results of KIP-258 whenever they are completed.
> https://github.com/apache/kafka/pull/5527
>
> Thanks,
>
> Adam

Is the HighWatermarkResolverProccessorsupplier already updated in the 
PR? expected it to change to Windowed<K>,Long Missing something?

>
>
>
> On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> Correction on my previous email: KAFKA-5533 is the wrong link, as it is for
>> corresponding changelog mechanisms. But as part of KIP-258 we do want to
>> have "handling out-of-order data for source KTable" such that instead of
>> blindly apply the updates to the materialized store, i.e. following offset
>> ordering, we will reject updates that are older than the current key's
>> timestamps, i.e. following timestamp ordering.
>>
>>
>> Guozhang
>>
>> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>
>>> Hello Adam,
>>>
>>> Thanks for the explanation. Regarding the final step (i.e. the high
>>> watermark store, now altered to be replaced with a window store), I think
>>> another current on-going KIP may actually help:
>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>>>
>>>
>>> This is for adding the timestamp into a key-value store (i.e. only for
>>> non-windowed KTable), and then one of its usage, as described in
>>> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
>>> "reject" updates from the source topics if its timestamp is smaller than
>>> the current key's latest update timestamp. I think it is very similar to
>>> what you have in mind for high watermark based filtering, while you only
>>> need to make sure that the timestamps of the joining records are
>> correctly
>>> inherited though the whole topology to the final stage.
>>>
>>> Note that this KIP is for key-value store and hence non-windowed KTables
>>> only, but for windowed KTables we do not really have a good support for
>>> their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107) I
>>> think we can just consider non-windowed KTable-KTable non-key joins for
>>> now. In which case, KIP-258 should help.
>>>
>>>
>>>
>>> Guozhang
>>>
>>>
>>>
>>> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>>
>>>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>>>
>>>>> Hi Guozhang
>>>>>
>>>>> Current highwater mark implementation would grow endlessly based on
>>>>> primary key of original event. It is a pair of (<this table primary
>> key>,
>>>>> <highest offset seen for that key>). This is used to differentiate
>> between
>>>>> late arrivals and new updates. My newest proposal would be to replace
>> it
>>>>> with a Windowed state store of Duration N. This would allow the same
>>>>> behaviour, but cap the size based on time. This should allow for all
>>>>> late-arriving events to be processed, and should be customizable by the
>>>>> user to tailor to their own needs (ie: perhaps just 10 minutes of
>> window,
>>>>> or perhaps 7 days...).
>>>>>
>>>> Hi Adam, using time based retention can do the trick here. Even if I
>>>> would still like to see the automatic repartitioning optional since I
>> would
>>>> just reshuffle again. With windowed store I am a little bit sceptical
>> about
>>>> how to determine the window. So esentially one could run into problems
>> when
>>>> the rapid change happens near a window border. I will check you
>>>> implementation in detail, if its problematic, we could still check _all_
>>>> windows on read with not to bad performance impact I guess. Will let you
>>>> know if the implementation would be correct as is. I wouldn't not like
>> to
>>>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I
>> think
>>>> we can't expect that.
>>>>
>>>>>
>>>>>
>>>>> @Jan
>>>>> I believe I understand what you mean now - thanks for the diagram, it
>>>>> did really help. You are correct that I do not have the original
>> primary
>>>>> key available, and I can see that if it was available then you would be
>>>>> able to add and remove events from the Map. That being said, I
>> encourage
>>>>> you to finish your diagrams / charts just for clarity for everyone
>> else.
>>>>>
>>>>> Yeah 100%, this giphy thing is just really hard work. But I understand
>>>> the benefits for the rest. Sorry about the original primary key, We have
>>>> join and Group by implemented our own in PAPI and basically not using
>> any
>>>> DSL (Just the abstraction). Completely missed that in original DSL its
>> not
>>>> there and just assumed it. total brain mess up on my end. Will finish
>> the
>>>> chart as soon as i get a quite evening this week.
>>>>
>>>> My follow up question for you is, won't the Map stay inside the State
>>>>> Store indefinitely after all of the changes have propagated? Isn't this
>>>>> effectively the same as a highwater mark state store?
>>>>>
>>>> Thing is that if the map is empty, substractor is gonna return `null`
>> and
>>>> the key is removed from the keyspace. But there is going to be a store
>>>> 100%, the good thing is that I can use this store directly for
>>>> materialize() / enableSendingOldValues() is a regular store, satisfying
>>>> all gurantees needed for further groupby / join. The Windowed store is
>> not
>>>> keeping the values, so for the next statefull operation we would
>>>> need to instantiate an extra store. or we have the window store also
>> have
>>>> the values then.
>>>>
>>>> Long story short. if we can flip in a custom group by before
>>>> repartitioning to the original primary key i think it would help the
>> users
>>>> big time in building efficient apps. Given the original primary key
>> issue I
>>>> understand that we do not have a solid foundation to build on.
>>>> Leaving primary key carry along to the user. very unfortunate. I could
>>>> understand the decision goes like that. I do not think its a good
>> decision.
>>>>
>>>>
>>>>>
>>>>>
>>>>>
>>>>> Thanks
>>>>> Adam
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>>>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
>> wrote:
>>>>>
>>>>>      please remove me from this group
>>>>>
>>>>>      On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>
>>>>>      wrote:
>>>>>
>>>>>      > Hi Adam,
>>>>>      >
>>>>>      > give me some time, will make such a chart. last time i didn't
>>>>>      get along
>>>>>      > well with giphy and ruined all your charts.
>>>>>      > Hopefully i can get it done today
>>>>>      >
>>>>>      > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>>>      > > Hi Jan
>>>>>      > >
>>>>>      > > I have included a diagram of what I attempted on the KIP.
>>>>>      > >
>>>>>      >
>>>>>      https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>>>> inKTable-GroupBy+Reduce/Aggregate
>>>>>      <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>>>      > >
>>>>>      > > I attempted this back at the start of my own implementation of
>>>>>      this
>>>>>      > > solution, and since I could not get it to work I have since
>>>>>      discarded the
>>>>>      > > code. At this point in time, if you wish to continue pursuing
>>>>>      for your
>>>>>      > > groupBy solution, I ask that you please create a diagram on
>>>>>      the KIP
>>>>>      > > carefully explaining your solution. Please feel free to use
>>>>>      the image I
>>>>>      > > just posted as a starting point. I am having trouble
>>>>>      understanding your
>>>>>      > > explanations but I think that a carefully constructed diagram
>>>>>      will clear
>>>>>      > up
>>>>>      > > any misunderstandings. Alternately, please post a
>>>>>      comprehensive PR with
>>>>>      > > your solution. I can only guess at what you mean, and since I
>>>>>      value my
>>>>>      > own
>>>>>      > > time as much as you value yours, I believe it is your
>>>>>      responsibility to
>>>>>      > > provide an implementation instead of me trying to guess.
>>>>>      > >
>>>>>      > > Adam
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > >
>>>>>      > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>
>>>>>      > > wrote:
>>>>>      > >
>>>>>      > >> Hi James,
>>>>>      > >>
>>>>>      > >> nice to see you beeing interested. kafka streams at this
>>>>>      point supports
>>>>>      > >> all sorts of joins as long as both streams have the same key.
>>>>>      > >> Adam is currently implementing a join where a KTable and a
>>>>>      KTable can
>>>>>      > have
>>>>>      > >> a one to many relation ship (1:n). We exploit that rocksdb is
>> a
>>>>>      > >> datastore that keeps data sorted (At least exposes an API to
>>>>>      access the
>>>>>      > >> stored data in a sorted fashion).
>>>>>      > >>
>>>>>      > >> I think the technical caveats are well understood now and we
>> are
>>>>>      > basically
>>>>>      > >> down to philosophy and API Design ( when Adam sees my newest
>>>>>      message).
>>>>>      > >> I have a lengthy track record of loosing those kinda
>>>>>      arguments within
>>>>>      > the
>>>>>      > >> streams community and I have no clue why. So I literally
>>>>>      can't wait for
>>>>>      > you
>>>>>      > >> to churn through this thread and give you opinion on how we
>>>>>      should
>>>>>      > design
>>>>>      > >> the return type of the oneToManyJoin and how many power we
>>>>>      want to give
>>>>>      > to
>>>>>      > >> the user vs "simplicity" (where simplicity isn't really that
>>>>>      as users
>>>>>      > still
>>>>>      > >> need to understand it I argue)
>>>>>      > >>
>>>>>      > >> waiting for you to join in on the discussion
>>>>>      > >>
>>>>>      > >> Best Jan
>>>>>      > >>
>>>>>      > >>
>>>>>      > >>
>>>>>      > >> On 07.09.2018 15:49, James Kwan wrote:
>>>>>      > >>
>>>>>      > >>> I am new to this group and I found this subject
>>>>>      interesting.  Sounds
>>>>>      > like
>>>>>      > >>> you guys want to implement a join table of two streams? Is
>>>>> there
>>>>>      > somewhere
>>>>>      > >>> I can see the original requirement or proposal?
>>>>>      > >>>
>>>>>      > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>>>      <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>
>>>>>      > >>>> wrote:
>>>>>      > >>>>
>>>>>      > >>>>
>>>>>      > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>>      > >>>>
>>>>>      > >>>>> I'm currently testing using a Windowed Store to store the
>>>>>      highwater
>>>>>      > >>>>> mark.
>>>>>      > >>>>> By all indications this should work fine, with the caveat
>>>>>      being that
>>>>>      > it
>>>>>      > >>>>> can
>>>>>      > >>>>> only resolve out-of-order arrival for up to the size of
>>>>>      the window
>>>>>      > (ie:
>>>>>      > >>>>> 24h, 72h, etc). This would remove the possibility of it
>> being
>>>>>      > unbounded
>>>>>      > >>>>> in
>>>>>      > >>>>> size.
>>>>>      > >>>>>
>>>>>      > >>>>> With regards to Jan's suggestion, I believe this is where
>>>>>      we will
>>>>>      > have
>>>>>      > >>>>> to
>>>>>      > >>>>> remain in disagreement. While I do not disagree with your
>>>>>      statement
>>>>>      > >>>>> about
>>>>>      > >>>>> there likely to be additional joins done in a real-world
>>>>>      workflow, I
>>>>>      > do
>>>>>      > >>>>> not
>>>>>      > >>>>> see how you can conclusively deal with out-of-order arrival
>>>>> of
>>>>>      > >>>>> foreign-key
>>>>>      > >>>>> changes and subsequent joins. I have attempted what I
>>>>>      think you have
>>>>>      > >>>>> proposed (without a high-water, using groupBy and reduce)
>>>>>      and found
>>>>>      > >>>>> that if
>>>>>      > >>>>> the foreign key changes too quickly, or the load on a
>>>>>      stream thread
>>>>>      > is
>>>>>      > >>>>> too
>>>>>      > >>>>> high, the joined messages will arrive out-of-order and be
>>>>>      incorrectly
>>>>>      > >>>>> propagated, such that an intermediate event is represented
>>>>>      as the
>>>>>      > final
>>>>>      > >>>>> event.
>>>>>      > >>>>>
>>>>>      > >>>> Can you shed some light on your groupBy implementation.
>>>>>      There must be
>>>>>      > >>>> some sort of flaw in it.
>>>>>      > >>>> I have a suspicion where it is, I would just like to
>>>>>      confirm. The idea
>>>>>      > >>>> is bullet proof and it must be
>>>>>      > >>>> an implementation mess up. I would like to clarify before
>>>>>      we draw a
>>>>>      > >>>> conclusion.
>>>>>      > >>>>
>>>>>      > >>>>    Repartitioning the scattered events back to their
>> original
>>>>>      > >>>>> partitions is the only way I know how to conclusively deal
>>>>>      with
>>>>>      > >>>>> out-of-order events in a given time frame, and to ensure
>>>>>      that the
>>>>>      > data
>>>>>      > >>>>> is
>>>>>      > >>>>> eventually consistent with the input events.
>>>>>      > >>>>>
>>>>>      > >>>>> If you have some code to share that illustrates your
>>>>>      approach, I
>>>>>      > would
>>>>>      > >>>>> be
>>>>>      > >>>>> very grateful as it would remove any misunderstandings
>>>>>      that I may
>>>>>      > have.
>>>>>      > >>>>>
>>>>>      > >>>> ah okay you were looking for my code. I don't have
>>>>>      something easily
>>>>>      > >>>> readable here as its bloated with OO-patterns.
>>>>>      > >>>>
>>>>>      > >>>> its anyhow trivial:
>>>>>      > >>>>
>>>>>      > >>>> @Override
>>>>>      > >>>>      public T apply(K aggKey, V value, T aggregate)
>>>>>      > >>>>      {
>>>>>      > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
>>>>>      imaginary
>>>>>      > >>>>          U toModifyKey = mapper.apply(value);
>>>>>      > >>>>              << this is the place where people actually
>>>>>      gonna have
>>>>>      > issues
>>>>>      > >>>> and why you probably couldn't do it. we would need to find
>>>>>      a solution
>>>>>      > here.
>>>>>      > >>>> I didn't realize that yet.
>>>>>      > >>>>              << we propagate the field in the joiner, so
>>>>>      that we can
>>>>>      > pick
>>>>>      > >>>> it up in an aggregate. Probably you have not thought of
>>>>>      this in your
>>>>>      > >>>> approach right?
>>>>>      > >>>>              << I am very open to find a generic solution
>>>>>      here. In my
>>>>>      > >>>> honest opinion this is broken in KTableImpl.GroupBy that it
>>>>>      looses
>>>>>      > the keys
>>>>>      > >>>> and only maintains the aggregate key.
>>>>>      > >>>>              << I abstracted it away back then way before i
>>>>> was
>>>>>      > thinking
>>>>>      > >>>> of oneToMany join. That is why I didn't realize its
>>>>>      significance here.
>>>>>      > >>>>              << Opinions?
>>>>>      > >>>>
>>>>>      > >>>>          for (V m : current)
>>>>>      > >>>>          {
>>>>>      > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>>>      > >>>>          }
>>>>>      > >>>>          if (isAdder)
>>>>>      > >>>>          {
>>>>>      > >>>> currentStateAsMap.put(toModifyKey, value);
>>>>>      > >>>>          }
>>>>>      > >>>>          else
>>>>>      > >>>>          {
>>>>>      > >>>> currentStateAsMap.remove(toModifyKey);
>>>>>      > >>>> if(currentStateAsMap.isEmpty()){
>>>>>      > >>>>                  return null;
>>>>>      > >>>>              }
>>>>>      > >>>>          }
>>>>>      > >>>>          retrun asAggregateType(currentStateAsMap)
>>>>>      > >>>>      }
>>>>>      > >>>>
>>>>>      > >>>>
>>>>>      > >>>>
>>>>>      > >>>>
>>>>>      > >>>>
>>>>>      > >>>> Thanks,
>>>>>      > >>>>> Adam
>>>>>      > >>>>>
>>>>>      > >>>>>
>>>>>      > >>>>>
>>>>>      > >>>>>
>>>>>      > >>>>>
>>>>>      > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>>>      > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>>>
>>>>>      > >>>>> wrote:
>>>>>      > >>>>>
>>>>>      > >>>>> Thanks Adam for bringing Matthias to speed!
>>>>>      > >>>>>> about the differences. I think re-keying back should be
>>>>>      optional at
>>>>>      > >>>>>> best.
>>>>>      > >>>>>> I would say we return a KScatteredTable with reshuffle()
>>>>>      returning
>>>>>      > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>>>      repartitioning
>>>>>      > >>>>>> optional.
>>>>>      > >>>>>> I am also in a big favour of doing the out of order
>>>>>      processing using
>>>>>      > >>>>>> group
>>>>>      > >>>>>> by instead high water mark tracking.
>>>>>      > >>>>>> Just because unbounded growth is just scary + It saves us
>>>>>      the header
>>>>>      > >>>>>> stuff.
>>>>>      > >>>>>>
>>>>>      > >>>>>> I think the abstraction of always repartitioning back is
>>>>>      just not so
>>>>>      > >>>>>> strong. Like the work has been done before we partition
>>>>>      back and
>>>>>      > >>>>>> grouping
>>>>>      > >>>>>> by something else afterwards is really common.
>>>>>      > >>>>>>
>>>>>      > >>>>>>
>>>>>      > >>>>>>
>>>>>      > >>>>>>
>>>>>      > >>>>>>
>>>>>      > >>>>>>
>>>>>      > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>      > >>>>>>
>>>>>      > >>>>>> Hi Matthias
>>>>>      > >>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> While name spacing would be possible, it would require to
>>>>>      > deserialize
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>      suggest to
>>>>>      > no
>>>>>      > >>>>>>>> namespace for now to avoid the overhead. If this
>> becomes a
>>>>>      > problem in
>>>>>      > >>>>>>>> the future, we can still add name spacing later on.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Agreed. I will go with using a reserved string and
>>>>>      document it.
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> My main concern about the design it the type of the
>>>>>      result KTable:
>>>>>      > If
>>>>>      > >>>>>>> I
>>>>>      > >>>>>>> understood the proposal correctly,
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> In your example, you have table1 and table2 swapped.
>>>>>      Here is how it
>>>>>      > >>>>>>> works
>>>>>      > >>>>>>> currently:
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> 1) table1 has the records that contain the foreign key
>>>>>      within their
>>>>>      > >>>>>>> value.
>>>>>      > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>>>>>      > >>>>>>> <c,(fk=B,bar=3)>
>>>>>      > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> 2) A Value mapper is required to extract the foreign key.
>>>>>      > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>>>      <http://value.fk> )
>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> The mapper is applied to each element in table1, and a
>>>>>      new combined
>>>>>      > >>>>>>> key is
>>>>>      > >>>>>>> made:
>>>>>      > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>,
>>>>>      <B-c,
>>>>>      > >>>>>>> (fk=B,bar=3)>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> a) Stream Thread with Partition 0:
>>>>>      > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>>>      (fk=A,bar=2)>
>>>>>      > >>>>>>> Table2: <A,X>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> b) Stream Thread with Partition 1:
>>>>>      > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>      > >>>>>>> Table2: <B,Y>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> 4) From here, they can be joined together locally by
>>>>>      applying the
>>>>>      > >>>>>>> joiner
>>>>>      > >>>>>>> function.
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> At this point, Jan's design and my design deviate. My
>>>>>      design goes
>>>>>      > on
>>>>>      > >>>>>>> to
>>>>>      > >>>>>>> repartition the data post-join and resolve out-of-order
>>>>>      arrival of
>>>>>      > >>>>>>> records,
>>>>>      > >>>>>>> finally returning the data keyed just the original key.
>>>>>      I do not
>>>>>      > >>>>>>> expose
>>>>>      > >>>>>>> the
>>>>>      > >>>>>>> CombinedKey or any of the internals outside of the
>>>>>      joinOnForeignKey
>>>>>      > >>>>>>> function. This does make for larger footprint, but it
>>>>>      removes all
>>>>>      > >>>>>>> agency
>>>>>      > >>>>>>> for resolving out-of-order arrivals and handling
>>>>>      CombinedKeys from
>>>>>      > the
>>>>>      > >>>>>>> user. I believe that this makes the function much easier
>>>>>      to use.
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> Let me know if this helps resolve your questions, and
>>>>>      please feel
>>>>>      > >>>>>>> free to
>>>>>      > >>>>>>> add anything else on your mind.
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> Thanks again,
>>>>>      > >>>>>>> Adam
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>      > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>>>>>
>>>>>      > >>>>>>> wrote:
>>>>>      > >>>>>>>
>>>>>      > >>>>>>> Hi,
>>>>>      > >>>>>>>
>>>>>      > >>>>>>>> I am just catching up on this thread. I did not read
>>>>>      everything so
>>>>>      > >>>>>>>> far,
>>>>>      > >>>>>>>> but want to share couple of initial thoughts:
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Headers: I think there is a fundamental difference
>>>>>      between header
>>>>>      > >>>>>>>> usage
>>>>>      > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>>>>>      changelog topic
>>>>>      > >>>>>>>> that
>>>>>      > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>>>>>      to write
>>>>>      > into
>>>>>      > >>>>>>>> them. In fact, no user header are written into the
>>>>>      changelog topic
>>>>>      > >>>>>>>> and
>>>>>      > >>>>>>>> thus, there are not conflicts.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Nevertheless, I don't see a big issue with using
>>>>>      headers within
>>>>>      > >>>>>>>> Streams.
>>>>>      > >>>>>>>> As long as we document it, we can have some "reserved"
>>>>>      header keys
>>>>>      > >>>>>>>> and
>>>>>      > >>>>>>>> users are not allowed to use when processing data with
>>>>>      Kafka
>>>>>      > Streams.
>>>>>      > >>>>>>>> IMHO, this should be ok.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> I think there is a safe way to avoid conflicts, since
>>>>> these
>>>>>      > headers
>>>>>      > >>>>>>>> are
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>> only needed in internal topics (I think):
>>>>>      > >>>>>>>>> For internal and changelog topics, we can namespace
>>>>>      all headers:
>>>>>      > >>>>>>>>> * user-defined headers are namespaced as "external." +
>>>>>      headerKey
>>>>>      > >>>>>>>>> * internal headers are namespaced as "internal." +
>>>>>      headerKey
>>>>>      > >>>>>>>>>
>>>>>      > >>>>>>>>> While name spacing would be possible, it would require
>> to
>>>>>      > >>>>>>>> deserialize
>>>>>      > >>>>>>>> user headers what implies a runtime overhead. I would
>>>>>      suggest to
>>>>>      > no
>>>>>      > >>>>>>>> namespace for now to avoid the overhead. If this
>> becomes a
>>>>>      > problem in
>>>>>      > >>>>>>>> the future, we can still add name spacing later on.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> My main concern about the design it the type of the
>>>>>      result KTable:
>>>>>      > >>>>>>>> If I
>>>>>      > >>>>>>>> understood the proposal correctly,
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> KTable<K1,V1> table1 = ...
>>>>>      > >>>>>>>> KTable<K2,V2> table2 = ...
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> implies that the `joinedTable` has the same key as the
>>>>>      left input
>>>>>      > >>>>>>>> table.
>>>>>      > >>>>>>>> IMHO, this does not work because if table2 contains
>>>>>      multiple rows
>>>>>      > >>>>>>>> that
>>>>>      > >>>>>>>> join with a record in table1 (what is the main purpose
>> of
>>>>> a
>>>>>      > foreign
>>>>>      > >>>>>>>> key
>>>>>      > >>>>>>>> join), the result table would only contain a single
>>>>>      join result,
>>>>>      > but
>>>>>      > >>>>>>>> not
>>>>>      > >>>>>>>> multiple.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Example:
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> table1 input stream: <A,X>
>>>>>      > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>>>>>      "A" joins).
>>>>>      > If
>>>>>      > >>>>>>>> the
>>>>>      > >>>>>>>> result key is the same key as key of table1, this
>>>>>      implies that the
>>>>>      > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>>>>>      but not
>>>>>      > both.
>>>>>      > >>>>>>>> Because the share the same key, whatever result record
>>>>>      we emit
>>>>>      > later,
>>>>>      > >>>>>>>> overwrite the previous result.
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> This is the reason why Jan originally proposed to use a
>>>>>      > combination
>>>>>      > >>>>>>>> of
>>>>>      > >>>>>>>> both primary keys of the input tables as key of the
>>>>>      output table.
>>>>>      > >>>>>>>> This
>>>>>      > >>>>>>>> makes the keys of the output table unique and we can
>>>>>      store both in
>>>>>      > >>>>>>>> the
>>>>>      > >>>>>>>> output table:
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Thoughts?
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> -Matthias
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>      > >>>>>>>>
>>>>>      > >>>>>>>> Just on remark here.
>>>>>      > >>>>>>>>> The high-watermark could be disregarded. The decision
>>>>>      about the
>>>>>      > >>>>>>>>> forward
>>>>>      > >>>>>>>>> depends on the size of the aggregated map.
>>>>>      > >>>>>>>>> Only 1 element long maps would be unpacked and
>>>>>      forwarded. 0
>>>>>      > element
>>>>>      > >>>>>>>>> maps
>>>>>      > >>>>>>>>> would be published as delete. Any other count
>>>>>      > >>>>>>>>> of map entries is in "waiting for correct deletes to
>>>>>      > arrive"-state.
>>>>>      > >>>>>>>>>
>>>>>      > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>      > >>>>>>>>>
>>>>>      > >>>>>>>>> It does look like I could replace the second
>>>>>      repartition store
>>>>>      > and
>>>>>      > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>>>>>      it looks
>>>>>      > like
>>>>>      > >>>>>>>>>> I
>>>>>      > >>>>>>>>>> would
>>>>>      > >>>>>>>>>> still need to store the highwater value within the
>>>>>      materialized
>>>>>      > >>>>>>>>>> store,
>>>>>      > >>>>>>>>>>
>>>>>      > >>>>>>>>>> to
>>>>>      > >>>>>>>>> compare the arrival of out-of-order records (assuming
>> my
>>>>>      > >>>>>>>>> understanding
>>>>>      > >>>>>>>>> of
>>>>>      > >>>>>>>>> THIS is correct...). This in effect is the same as the
>>>>>      design I
>>>>>      > have
>>>>>      > >>>>>>>>> now,
>>>>>      > >>>>>>>>> just with the two tables merged together.
>>>>>      > >>>>>>>>>
>>>>>      >
>>>>>      >
>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>
>>
>>
>> --
>> -- Guozhang
>>
>


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
@Guozhang

Thanks for the information. This is indeed something that will be extremely
useful for this KIP.

@Jan
Thanks for your explanations. That being said, I will not be moving ahead
with an implementation using reshuffle/groupBy solution as you propose.
That being said, if you wish to implement it yourself off of my current PR
and submit it as a competitive alternative, I would be more than happy to
help vet that as an alternate solution. As it stands right now, I do not
really have more time to invest into alternatives without there being a
strong indication from the binding voters which they would prefer.


I will look at finishing up my PR with the windowed state store in the next
week or so, exercising it via tests, and then I will come back for final
discussions. In the meantime, I hope that any of the binding voters could
take a look at the KIP in the wiki. I have updated it according to the
latest plan:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable

I have also updated the KIP PR to use a windowed store. This could be
replaced by the results of KIP-258 whenever they are completed.
https://github.com/apache/kafka/pull/5527

Thanks,

Adam



On Fri, Sep 14, 2018 at 2:24 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Correction on my previous email: KAFKA-5533 is the wrong link, as it is for
> corresponding changelog mechanisms. But as part of KIP-258 we do want to
> have "handling out-of-order data for source KTable" such that instead of
> blindly apply the updates to the materialized store, i.e. following offset
> ordering, we will reject updates that are older than the current key's
> timestamps, i.e. following timestamp ordering.
>
>
> Guozhang
>
> On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > Hello Adam,
> >
> > Thanks for the explanation. Regarding the final step (i.e. the high
> > watermark store, now altered to be replaced with a window store), I think
> > another current on-going KIP may actually help:
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
> >
> >
> > This is for adding the timestamp into a key-value store (i.e. only for
> > non-windowed KTable), and then one of its usage, as described in
> > https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
> > "reject" updates from the source topics if its timestamp is smaller than
> > the current key's latest update timestamp. I think it is very similar to
> > what you have in mind for high watermark based filtering, while you only
> > need to make sure that the timestamps of the joining records are
> correctly
> > inherited though the whole topology to the final stage.
> >
> > Note that this KIP is for key-value store and hence non-windowed KTables
> > only, but for windowed KTables we do not really have a good support for
> > their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107) I
> > think we can just consider non-windowed KTable-KTable non-key joins for
> > now. In which case, KIP-258 should help.
> >
> >
> >
> > Guozhang
> >
> >
> >
> > On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >>
> >> On 11.09.2018 18:00, Adam Bellemare wrote:
> >>
> >>> Hi Guozhang
> >>>
> >>> Current highwater mark implementation would grow endlessly based on
> >>> primary key of original event. It is a pair of (<this table primary
> key>,
> >>> <highest offset seen for that key>). This is used to differentiate
> between
> >>> late arrivals and new updates. My newest proposal would be to replace
> it
> >>> with a Windowed state store of Duration N. This would allow the same
> >>> behaviour, but cap the size based on time. This should allow for all
> >>> late-arriving events to be processed, and should be customizable by the
> >>> user to tailor to their own needs (ie: perhaps just 10 minutes of
> window,
> >>> or perhaps 7 days...).
> >>>
> >> Hi Adam, using time based retention can do the trick here. Even if I
> >> would still like to see the automatic repartitioning optional since I
> would
> >> just reshuffle again. With windowed store I am a little bit sceptical
> about
> >> how to determine the window. So esentially one could run into problems
> when
> >> the rapid change happens near a window border. I will check you
> >> implementation in detail, if its problematic, we could still check _all_
> >> windows on read with not to bad performance impact I guess. Will let you
> >> know if the implementation would be correct as is. I wouldn't not like
> to
> >> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I
> think
> >> we can't expect that.
> >>
> >>>
> >>>
> >>> @Jan
> >>> I believe I understand what you mean now - thanks for the diagram, it
> >>> did really help. You are correct that I do not have the original
> primary
> >>> key available, and I can see that if it was available then you would be
> >>> able to add and remove events from the Map. That being said, I
> encourage
> >>> you to finish your diagrams / charts just for clarity for everyone
> else.
> >>>
> >>> Yeah 100%, this giphy thing is just really hard work. But I understand
> >> the benefits for the rest. Sorry about the original primary key, We have
> >> join and Group by implemented our own in PAPI and basically not using
> any
> >> DSL (Just the abstraction). Completely missed that in original DSL its
> not
> >> there and just assumed it. total brain mess up on my end. Will finish
> the
> >> chart as soon as i get a quite evening this week.
> >>
> >> My follow up question for you is, won't the Map stay inside the State
> >>> Store indefinitely after all of the changes have propagated? Isn't this
> >>> effectively the same as a highwater mark state store?
> >>>
> >> Thing is that if the map is empty, substractor is gonna return `null`
> and
> >> the key is removed from the keyspace. But there is going to be a store
> >> 100%, the good thing is that I can use this store directly for
> >> materialize() / enableSendingOldValues() is a regular store, satisfying
> >> all gurantees needed for further groupby / join. The Windowed store is
> not
> >> keeping the values, so for the next statefull operation we would
> >> need to instantiate an extra store. or we have the window store also
> have
> >> the values then.
> >>
> >> Long story short. if we can flip in a custom group by before
> >> repartitioning to the original primary key i think it would help the
> users
> >> big time in building efficient apps. Given the original primary key
> issue I
> >> understand that we do not have a solid foundation to build on.
> >> Leaving primary key carry along to the user. very unfortunate. I could
> >> understand the decision goes like that. I do not think its a good
> decision.
> >>
> >>
> >>>
> >>>
> >>>
> >>> Thanks
> >>> Adam
> >>>
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
> >>> dumbreprajakta311@gmail.com <ma...@gmail.com>>
> wrote:
> >>>
> >>>     please remove me from this group
> >>>
> >>>     On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
> >>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>
> >>>     wrote:
> >>>
> >>>     > Hi Adam,
> >>>     >
> >>>     > give me some time, will make such a chart. last time i didn't
> >>>     get along
> >>>     > well with giphy and ruined all your charts.
> >>>     > Hopefully i can get it done today
> >>>     >
> >>>     > On 08.09.2018 16:00, Adam Bellemare wrote:
> >>>     > > Hi Jan
> >>>     > >
> >>>     > > I have included a diagram of what I attempted on the KIP.
> >>>     > >
> >>>     >
> >>>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
> >>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
> >>> inKTable-GroupBy+Reduce/Aggregate
> >>>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
> >>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
> >>> ginKTable-GroupBy+Reduce/Aggregate>
> >>>     > >
> >>>     > > I attempted this back at the start of my own implementation of
> >>>     this
> >>>     > > solution, and since I could not get it to work I have since
> >>>     discarded the
> >>>     > > code. At this point in time, if you wish to continue pursuing
> >>>     for your
> >>>     > > groupBy solution, I ask that you please create a diagram on
> >>>     the KIP
> >>>     > > carefully explaining your solution. Please feel free to use
> >>>     the image I
> >>>     > > just posted as a starting point. I am having trouble
> >>>     understanding your
> >>>     > > explanations but I think that a carefully constructed diagram
> >>>     will clear
> >>>     > up
> >>>     > > any misunderstandings. Alternately, please post a
> >>>     comprehensive PR with
> >>>     > > your solution. I can only guess at what you mean, and since I
> >>>     value my
> >>>     > own
> >>>     > > time as much as you value yours, I believe it is your
> >>>     responsibility to
> >>>     > > provide an implementation instead of me trying to guess.
> >>>     > >
> >>>     > > Adam
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > >
> >>>     > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
> >>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>
> >>>     > > wrote:
> >>>     > >
> >>>     > >> Hi James,
> >>>     > >>
> >>>     > >> nice to see you beeing interested. kafka streams at this
> >>>     point supports
> >>>     > >> all sorts of joins as long as both streams have the same key.
> >>>     > >> Adam is currently implementing a join where a KTable and a
> >>>     KTable can
> >>>     > have
> >>>     > >> a one to many relation ship (1:n). We exploit that rocksdb is
> a
> >>>     > >> datastore that keeps data sorted (At least exposes an API to
> >>>     access the
> >>>     > >> stored data in a sorted fashion).
> >>>     > >>
> >>>     > >> I think the technical caveats are well understood now and we
> are
> >>>     > basically
> >>>     > >> down to philosophy and API Design ( when Adam sees my newest
> >>>     message).
> >>>     > >> I have a lengthy track record of loosing those kinda
> >>>     arguments within
> >>>     > the
> >>>     > >> streams community and I have no clue why. So I literally
> >>>     can't wait for
> >>>     > you
> >>>     > >> to churn through this thread and give you opinion on how we
> >>>     should
> >>>     > design
> >>>     > >> the return type of the oneToManyJoin and how many power we
> >>>     want to give
> >>>     > to
> >>>     > >> the user vs "simplicity" (where simplicity isn't really that
> >>>     as users
> >>>     > still
> >>>     > >> need to understand it I argue)
> >>>     > >>
> >>>     > >> waiting for you to join in on the discussion
> >>>     > >>
> >>>     > >> Best Jan
> >>>     > >>
> >>>     > >>
> >>>     > >>
> >>>     > >> On 07.09.2018 15:49, James Kwan wrote:
> >>>     > >>
> >>>     > >>> I am new to this group and I found this subject
> >>>     interesting.  Sounds
> >>>     > like
> >>>     > >>> you guys want to implement a join table of two streams? Is
> >>> there
> >>>     > somewhere
> >>>     > >>> I can see the original requirement or proposal?
> >>>     > >>>
> >>>     > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
> >>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>
> >>>     > >>>> wrote:
> >>>     > >>>>
> >>>     > >>>>
> >>>     > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>     > >>>>
> >>>     > >>>>> I'm currently testing using a Windowed Store to store the
> >>>     highwater
> >>>     > >>>>> mark.
> >>>     > >>>>> By all indications this should work fine, with the caveat
> >>>     being that
> >>>     > it
> >>>     > >>>>> can
> >>>     > >>>>> only resolve out-of-order arrival for up to the size of
> >>>     the window
> >>>     > (ie:
> >>>     > >>>>> 24h, 72h, etc). This would remove the possibility of it
> being
> >>>     > unbounded
> >>>     > >>>>> in
> >>>     > >>>>> size.
> >>>     > >>>>>
> >>>     > >>>>> With regards to Jan's suggestion, I believe this is where
> >>>     we will
> >>>     > have
> >>>     > >>>>> to
> >>>     > >>>>> remain in disagreement. While I do not disagree with your
> >>>     statement
> >>>     > >>>>> about
> >>>     > >>>>> there likely to be additional joins done in a real-world
> >>>     workflow, I
> >>>     > do
> >>>     > >>>>> not
> >>>     > >>>>> see how you can conclusively deal with out-of-order arrival
> >>> of
> >>>     > >>>>> foreign-key
> >>>     > >>>>> changes and subsequent joins. I have attempted what I
> >>>     think you have
> >>>     > >>>>> proposed (without a high-water, using groupBy and reduce)
> >>>     and found
> >>>     > >>>>> that if
> >>>     > >>>>> the foreign key changes too quickly, or the load on a
> >>>     stream thread
> >>>     > is
> >>>     > >>>>> too
> >>>     > >>>>> high, the joined messages will arrive out-of-order and be
> >>>     incorrectly
> >>>     > >>>>> propagated, such that an intermediate event is represented
> >>>     as the
> >>>     > final
> >>>     > >>>>> event.
> >>>     > >>>>>
> >>>     > >>>> Can you shed some light on your groupBy implementation.
> >>>     There must be
> >>>     > >>>> some sort of flaw in it.
> >>>     > >>>> I have a suspicion where it is, I would just like to
> >>>     confirm. The idea
> >>>     > >>>> is bullet proof and it must be
> >>>     > >>>> an implementation mess up. I would like to clarify before
> >>>     we draw a
> >>>     > >>>> conclusion.
> >>>     > >>>>
> >>>     > >>>>    Repartitioning the scattered events back to their
> original
> >>>     > >>>>> partitions is the only way I know how to conclusively deal
> >>>     with
> >>>     > >>>>> out-of-order events in a given time frame, and to ensure
> >>>     that the
> >>>     > data
> >>>     > >>>>> is
> >>>     > >>>>> eventually consistent with the input events.
> >>>     > >>>>>
> >>>     > >>>>> If you have some code to share that illustrates your
> >>>     approach, I
> >>>     > would
> >>>     > >>>>> be
> >>>     > >>>>> very grateful as it would remove any misunderstandings
> >>>     that I may
> >>>     > have.
> >>>     > >>>>>
> >>>     > >>>> ah okay you were looking for my code. I don't have
> >>>     something easily
> >>>     > >>>> readable here as its bloated with OO-patterns.
> >>>     > >>>>
> >>>     > >>>> its anyhow trivial:
> >>>     > >>>>
> >>>     > >>>> @Override
> >>>     > >>>>      public T apply(K aggKey, V value, T aggregate)
> >>>     > >>>>      {
> >>>     > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
> >>>     imaginary
> >>>     > >>>>          U toModifyKey = mapper.apply(value);
> >>>     > >>>>              << this is the place where people actually
> >>>     gonna have
> >>>     > issues
> >>>     > >>>> and why you probably couldn't do it. we would need to find
> >>>     a solution
> >>>     > here.
> >>>     > >>>> I didn't realize that yet.
> >>>     > >>>>              << we propagate the field in the joiner, so
> >>>     that we can
> >>>     > pick
> >>>     > >>>> it up in an aggregate. Probably you have not thought of
> >>>     this in your
> >>>     > >>>> approach right?
> >>>     > >>>>              << I am very open to find a generic solution
> >>>     here. In my
> >>>     > >>>> honest opinion this is broken in KTableImpl.GroupBy that it
> >>>     looses
> >>>     > the keys
> >>>     > >>>> and only maintains the aggregate key.
> >>>     > >>>>              << I abstracted it away back then way before i
> >>> was
> >>>     > thinking
> >>>     > >>>> of oneToMany join. That is why I didn't realize its
> >>>     significance here.
> >>>     > >>>>              << Opinions?
> >>>     > >>>>
> >>>     > >>>>          for (V m : current)
> >>>     > >>>>          {
> >>>     > >>>> currentStateAsMap.put(mapper.apply(m), m);
> >>>     > >>>>          }
> >>>     > >>>>          if (isAdder)
> >>>     > >>>>          {
> >>>     > >>>> currentStateAsMap.put(toModifyKey, value);
> >>>     > >>>>          }
> >>>     > >>>>          else
> >>>     > >>>>          {
> >>>     > >>>> currentStateAsMap.remove(toModifyKey);
> >>>     > >>>> if(currentStateAsMap.isEmpty()){
> >>>     > >>>>                  return null;
> >>>     > >>>>              }
> >>>     > >>>>          }
> >>>     > >>>>          retrun asAggregateType(currentStateAsMap)
> >>>     > >>>>      }
> >>>     > >>>>
> >>>     > >>>>
> >>>     > >>>>
> >>>     > >>>>
> >>>     > >>>>
> >>>     > >>>> Thanks,
> >>>     > >>>>> Adam
> >>>     > >>>>>
> >>>     > >>>>>
> >>>     > >>>>>
> >>>     > >>>>>
> >>>     > >>>>>
> >>>     > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> >>>     > Jan.Filipiak@trivago.com <ma...@trivago.com>>
> >>>
> >>>     > >>>>> wrote:
> >>>     > >>>>>
> >>>     > >>>>> Thanks Adam for bringing Matthias to speed!
> >>>     > >>>>>> about the differences. I think re-keying back should be
> >>>     optional at
> >>>     > >>>>>> best.
> >>>     > >>>>>> I would say we return a KScatteredTable with reshuffle()
> >>>     returning
> >>>     > >>>>>> KTable<originalKey,Joined> to make the backwards
> >>>     repartitioning
> >>>     > >>>>>> optional.
> >>>     > >>>>>> I am also in a big favour of doing the out of order
> >>>     processing using
> >>>     > >>>>>> group
> >>>     > >>>>>> by instead high water mark tracking.
> >>>     > >>>>>> Just because unbounded growth is just scary + It saves us
> >>>     the header
> >>>     > >>>>>> stuff.
> >>>     > >>>>>>
> >>>     > >>>>>> I think the abstraction of always repartitioning back is
> >>>     just not so
> >>>     > >>>>>> strong. Like the work has been done before we partition
> >>>     back and
> >>>     > >>>>>> grouping
> >>>     > >>>>>> by something else afterwards is really common.
> >>>     > >>>>>>
> >>>     > >>>>>>
> >>>     > >>>>>>
> >>>     > >>>>>>
> >>>     > >>>>>>
> >>>     > >>>>>>
> >>>     > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>     > >>>>>>
> >>>     > >>>>>> Hi Matthias
> >>>     > >>>>>>> Thank you for your feedback, I do appreciate it!
> >>>     > >>>>>>>
> >>>     > >>>>>>> While name spacing would be possible, it would require to
> >>>     > deserialize
> >>>     > >>>>>>>
> >>>     > >>>>>>>> user headers what implies a runtime overhead. I would
> >>>     suggest to
> >>>     > no
> >>>     > >>>>>>>> namespace for now to avoid the overhead. If this
> becomes a
> >>>     > problem in
> >>>     > >>>>>>>> the future, we can still add name spacing later on.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Agreed. I will go with using a reserved string and
> >>>     document it.
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>> My main concern about the design it the type of the
> >>>     result KTable:
> >>>     > If
> >>>     > >>>>>>> I
> >>>     > >>>>>>> understood the proposal correctly,
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>> In your example, you have table1 and table2 swapped.
> >>>     Here is how it
> >>>     > >>>>>>> works
> >>>     > >>>>>>> currently:
> >>>     > >>>>>>>
> >>>     > >>>>>>> 1) table1 has the records that contain the foreign key
> >>>     within their
> >>>     > >>>>>>> value.
> >>>     > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
> >>>     > >>>>>>> <c,(fk=B,bar=3)>
> >>>     > >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>     > >>>>>>>
> >>>     > >>>>>>> 2) A Value mapper is required to extract the foreign key.
> >>>     > >>>>>>> table1 foreign key mapper: ( value => value.fk
> >>>     <http://value.fk> )
> >>>
> >>>     > >>>>>>>
> >>>     > >>>>>>> The mapper is applied to each element in table1, and a
> >>>     new combined
> >>>     > >>>>>>> key is
> >>>     > >>>>>>> made:
> >>>     > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>,
> >>>     <B-c,
> >>>     > >>>>>>> (fk=B,bar=3)>
> >>>     > >>>>>>>
> >>>     > >>>>>>> 3) The rekeyed events are copartitioned with table2:
> >>>     > >>>>>>>
> >>>     > >>>>>>> a) Stream Thread with Partition 0:
> >>>     > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
> >>>     (fk=A,bar=2)>
> >>>     > >>>>>>> Table2: <A,X>
> >>>     > >>>>>>>
> >>>     > >>>>>>> b) Stream Thread with Partition 1:
> >>>     > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>     > >>>>>>> Table2: <B,Y>
> >>>     > >>>>>>>
> >>>     > >>>>>>> 4) From here, they can be joined together locally by
> >>>     applying the
> >>>     > >>>>>>> joiner
> >>>     > >>>>>>> function.
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>> At this point, Jan's design and my design deviate. My
> >>>     design goes
> >>>     > on
> >>>     > >>>>>>> to
> >>>     > >>>>>>> repartition the data post-join and resolve out-of-order
> >>>     arrival of
> >>>     > >>>>>>> records,
> >>>     > >>>>>>> finally returning the data keyed just the original key.
> >>>     I do not
> >>>     > >>>>>>> expose
> >>>     > >>>>>>> the
> >>>     > >>>>>>> CombinedKey or any of the internals outside of the
> >>>     joinOnForeignKey
> >>>     > >>>>>>> function. This does make for larger footprint, but it
> >>>     removes all
> >>>     > >>>>>>> agency
> >>>     > >>>>>>> for resolving out-of-order arrivals and handling
> >>>     CombinedKeys from
> >>>     > the
> >>>     > >>>>>>> user. I believe that this makes the function much easier
> >>>     to use.
> >>>     > >>>>>>>
> >>>     > >>>>>>> Let me know if this helps resolve your questions, and
> >>>     please feel
> >>>     > >>>>>>> free to
> >>>     > >>>>>>> add anything else on your mind.
> >>>     > >>>>>>>
> >>>     > >>>>>>> Thanks again,
> >>>     > >>>>>>> Adam
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>>
> >>>     > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> >>>     > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
> >>>
> >>>     > >>>>>>> wrote:
> >>>     > >>>>>>>
> >>>     > >>>>>>> Hi,
> >>>     > >>>>>>>
> >>>     > >>>>>>>> I am just catching up on this thread. I did not read
> >>>     everything so
> >>>     > >>>>>>>> far,
> >>>     > >>>>>>>> but want to share couple of initial thoughts:
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Headers: I think there is a fundamental difference
> >>>     between header
> >>>     > >>>>>>>> usage
> >>>     > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
> >>>     changelog topic
> >>>     > >>>>>>>> that
> >>>     > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
> >>>     to write
> >>>     > into
> >>>     > >>>>>>>> them. In fact, no user header are written into the
> >>>     changelog topic
> >>>     > >>>>>>>> and
> >>>     > >>>>>>>> thus, there are not conflicts.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Nevertheless, I don't see a big issue with using
> >>>     headers within
> >>>     > >>>>>>>> Streams.
> >>>     > >>>>>>>> As long as we document it, we can have some "reserved"
> >>>     header keys
> >>>     > >>>>>>>> and
> >>>     > >>>>>>>> users are not allowed to use when processing data with
> >>>     Kafka
> >>>     > Streams.
> >>>     > >>>>>>>> IMHO, this should be ok.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> I think there is a safe way to avoid conflicts, since
> >>> these
> >>>     > headers
> >>>     > >>>>>>>> are
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>> only needed in internal topics (I think):
> >>>     > >>>>>>>>> For internal and changelog topics, we can namespace
> >>>     all headers:
> >>>     > >>>>>>>>> * user-defined headers are namespaced as "external." +
> >>>     headerKey
> >>>     > >>>>>>>>> * internal headers are namespaced as "internal." +
> >>>     headerKey
> >>>     > >>>>>>>>>
> >>>     > >>>>>>>>> While name spacing would be possible, it would require
> to
> >>>     > >>>>>>>> deserialize
> >>>     > >>>>>>>> user headers what implies a runtime overhead. I would
> >>>     suggest to
> >>>     > no
> >>>     > >>>>>>>> namespace for now to avoid the overhead. If this
> becomes a
> >>>     > problem in
> >>>     > >>>>>>>> the future, we can still add name spacing later on.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> My main concern about the design it the type of the
> >>>     result KTable:
> >>>     > >>>>>>>> If I
> >>>     > >>>>>>>> understood the proposal correctly,
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> KTable<K1,V1> table1 = ...
> >>>     > >>>>>>>> KTable<K2,V2> table2 = ...
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> implies that the `joinedTable` has the same key as the
> >>>     left input
> >>>     > >>>>>>>> table.
> >>>     > >>>>>>>> IMHO, this does not work because if table2 contains
> >>>     multiple rows
> >>>     > >>>>>>>> that
> >>>     > >>>>>>>> join with a record in table1 (what is the main purpose
> of
> >>> a
> >>>     > foreign
> >>>     > >>>>>>>> key
> >>>     > >>>>>>>> join), the result table would only contain a single
> >>>     join result,
> >>>     > but
> >>>     > >>>>>>>> not
> >>>     > >>>>>>>> multiple.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Example:
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> table1 input stream: <A,X>
> >>>     > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
> >>>     "A" joins).
> >>>     > If
> >>>     > >>>>>>>> the
> >>>     > >>>>>>>> result key is the same key as key of table1, this
> >>>     implies that the
> >>>     > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
> >>>     but not
> >>>     > both.
> >>>     > >>>>>>>> Because the share the same key, whatever result record
> >>>     we emit
> >>>     > later,
> >>>     > >>>>>>>> overwrite the previous result.
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> This is the reason why Jan originally proposed to use a
> >>>     > combination
> >>>     > >>>>>>>> of
> >>>     > >>>>>>>> both primary keys of the input tables as key of the
> >>>     output table.
> >>>     > >>>>>>>> This
> >>>     > >>>>>>>> makes the keys of the output table unique and we can
> >>>     store both in
> >>>     > >>>>>>>> the
> >>>     > >>>>>>>> output table:
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Thoughts?
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> -Matthias
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>     > >>>>>>>>
> >>>     > >>>>>>>> Just on remark here.
> >>>     > >>>>>>>>> The high-watermark could be disregarded. The decision
> >>>     about the
> >>>     > >>>>>>>>> forward
> >>>     > >>>>>>>>> depends on the size of the aggregated map.
> >>>     > >>>>>>>>> Only 1 element long maps would be unpacked and
> >>>     forwarded. 0
> >>>     > element
> >>>     > >>>>>>>>> maps
> >>>     > >>>>>>>>> would be published as delete. Any other count
> >>>     > >>>>>>>>> of map entries is in "waiting for correct deletes to
> >>>     > arrive"-state.
> >>>     > >>>>>>>>>
> >>>     > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>     > >>>>>>>>>
> >>>     > >>>>>>>>> It does look like I could replace the second
> >>>     repartition store
> >>>     > and
> >>>     > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
> >>>     it looks
> >>>     > like
> >>>     > >>>>>>>>>> I
> >>>     > >>>>>>>>>> would
> >>>     > >>>>>>>>>> still need to store the highwater value within the
> >>>     materialized
> >>>     > >>>>>>>>>> store,
> >>>     > >>>>>>>>>>
> >>>     > >>>>>>>>>> to
> >>>     > >>>>>>>>> compare the arrival of out-of-order records (assuming
> my
> >>>     > >>>>>>>>> understanding
> >>>     > >>>>>>>>> of
> >>>     > >>>>>>>>> THIS is correct...). This in effect is the same as the
> >>>     design I
> >>>     > have
> >>>     > >>>>>>>>> now,
> >>>     > >>>>>>>>> just with the two tables merged together.
> >>>     > >>>>>>>>>
> >>>     >
> >>>     >
> >>>
> >>>
> >>>
> >>
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Guozhang Wang <wa...@gmail.com>.
Correction on my previous email: KAFKA-5533 is the wrong link, as it is for
corresponding changelog mechanisms. But as part of KIP-258 we do want to
have "handling out-of-order data for source KTable" such that instead of
blindly apply the updates to the materialized store, i.e. following offset
ordering, we will reject updates that are older than the current key's
timestamps, i.e. following timestamp ordering.


Guozhang

On Fri, Sep 14, 2018 at 11:21 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hello Adam,
>
> Thanks for the explanation. Regarding the final step (i.e. the high
> watermark store, now altered to be replaced with a window store), I think
> another current on-going KIP may actually help:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB
>
>
> This is for adding the timestamp into a key-value store (i.e. only for
> non-windowed KTable), and then one of its usage, as described in
> https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
> "reject" updates from the source topics if its timestamp is smaller than
> the current key's latest update timestamp. I think it is very similar to
> what you have in mind for high watermark based filtering, while you only
> need to make sure that the timestamps of the joining records are correctly
> inherited though the whole topology to the final stage.
>
> Note that this KIP is for key-value store and hence non-windowed KTables
> only, but for windowed KTables we do not really have a good support for
> their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107) I
> think we can just consider non-windowed KTable-KTable non-key joins for
> now. In which case, KIP-258 should help.
>
>
>
> Guozhang
>
>
>
> On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>>
>> On 11.09.2018 18:00, Adam Bellemare wrote:
>>
>>> Hi Guozhang
>>>
>>> Current highwater mark implementation would grow endlessly based on
>>> primary key of original event. It is a pair of (<this table primary key>,
>>> <highest offset seen for that key>). This is used to differentiate between
>>> late arrivals and new updates. My newest proposal would be to replace it
>>> with a Windowed state store of Duration N. This would allow the same
>>> behaviour, but cap the size based on time. This should allow for all
>>> late-arriving events to be processed, and should be customizable by the
>>> user to tailor to their own needs (ie: perhaps just 10 minutes of window,
>>> or perhaps 7 days...).
>>>
>> Hi Adam, using time based retention can do the trick here. Even if I
>> would still like to see the automatic repartitioning optional since I would
>> just reshuffle again. With windowed store I am a little bit sceptical about
>> how to determine the window. So esentially one could run into problems when
>> the rapid change happens near a window border. I will check you
>> implementation in detail, if its problematic, we could still check _all_
>> windows on read with not to bad performance impact I guess. Will let you
>> know if the implementation would be correct as is. I wouldn't not like to
>> assume that: offset(A) < offset(B) => timestamp(A)  < timestamp(B). I think
>> we can't expect that.
>>
>>>
>>>
>>> @Jan
>>> I believe I understand what you mean now - thanks for the diagram, it
>>> did really help. You are correct that I do not have the original primary
>>> key available, and I can see that if it was available then you would be
>>> able to add and remove events from the Map. That being said, I encourage
>>> you to finish your diagrams / charts just for clarity for everyone else.
>>>
>>> Yeah 100%, this giphy thing is just really hard work. But I understand
>> the benefits for the rest. Sorry about the original primary key, We have
>> join and Group by implemented our own in PAPI and basically not using any
>> DSL (Just the abstraction). Completely missed that in original DSL its not
>> there and just assumed it. total brain mess up on my end. Will finish the
>> chart as soon as i get a quite evening this week.
>>
>> My follow up question for you is, won't the Map stay inside the State
>>> Store indefinitely after all of the changes have propagated? Isn't this
>>> effectively the same as a highwater mark state store?
>>>
>> Thing is that if the map is empty, substractor is gonna return `null` and
>> the key is removed from the keyspace. But there is going to be a store
>> 100%, the good thing is that I can use this store directly for
>> materialize() / enableSendingOldValues() is a regular store, satisfying
>> all gurantees needed for further groupby / join. The Windowed store is not
>> keeping the values, so for the next statefull operation we would
>> need to instantiate an extra store. or we have the window store also have
>> the values then.
>>
>> Long story short. if we can flip in a custom group by before
>> repartitioning to the original primary key i think it would help the users
>> big time in building efficient apps. Given the original primary key issue I
>> understand that we do not have a solid foundation to build on.
>> Leaving primary key carry along to the user. very unfortunate. I could
>> understand the decision goes like that. I do not think its a good decision.
>>
>>
>>>
>>>
>>>
>>> Thanks
>>> Adam
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>>> dumbreprajakta311@gmail.com <ma...@gmail.com>> wrote:
>>>
>>>     please remove me from this group
>>>
>>>     On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>
>>>     wrote:
>>>
>>>     > Hi Adam,
>>>     >
>>>     > give me some time, will make such a chart. last time i didn't
>>>     get along
>>>     > well with giphy and ruined all your charts.
>>>     > Hopefully i can get it done today
>>>     >
>>>     > On 08.09.2018 16:00, Adam Bellemare wrote:
>>>     > > Hi Jan
>>>     > >
>>>     > > I have included a diagram of what I attempted on the KIP.
>>>     > >
>>>     >
>>>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Su
>>> pport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoining
>>> inKTable-GroupBy+Reduce/Aggregate
>>>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+S
>>> upport+non-key+joining+in+KTable#KIP-213Supportnon-keyjoinin
>>> ginKTable-GroupBy+Reduce/Aggregate>
>>>     > >
>>>     > > I attempted this back at the start of my own implementation of
>>>     this
>>>     > > solution, and since I could not get it to work I have since
>>>     discarded the
>>>     > > code. At this point in time, if you wish to continue pursuing
>>>     for your
>>>     > > groupBy solution, I ask that you please create a diagram on
>>>     the KIP
>>>     > > carefully explaining your solution. Please feel free to use
>>>     the image I
>>>     > > just posted as a starting point. I am having trouble
>>>     understanding your
>>>     > > explanations but I think that a carefully constructed diagram
>>>     will clear
>>>     > up
>>>     > > any misunderstandings. Alternately, please post a
>>>     comprehensive PR with
>>>     > > your solution. I can only guess at what you mean, and since I
>>>     value my
>>>     > own
>>>     > > time as much as you value yours, I believe it is your
>>>     responsibility to
>>>     > > provide an implementation instead of me trying to guess.
>>>     > >
>>>     > > Adam
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > >
>>>     > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>
>>>     > > wrote:
>>>     > >
>>>     > >> Hi James,
>>>     > >>
>>>     > >> nice to see you beeing interested. kafka streams at this
>>>     point supports
>>>     > >> all sorts of joins as long as both streams have the same key.
>>>     > >> Adam is currently implementing a join where a KTable and a
>>>     KTable can
>>>     > have
>>>     > >> a one to many relation ship (1:n). We exploit that rocksdb is a
>>>     > >> datastore that keeps data sorted (At least exposes an API to
>>>     access the
>>>     > >> stored data in a sorted fashion).
>>>     > >>
>>>     > >> I think the technical caveats are well understood now and we are
>>>     > basically
>>>     > >> down to philosophy and API Design ( when Adam sees my newest
>>>     message).
>>>     > >> I have a lengthy track record of loosing those kinda
>>>     arguments within
>>>     > the
>>>     > >> streams community and I have no clue why. So I literally
>>>     can't wait for
>>>     > you
>>>     > >> to churn through this thread and give you opinion on how we
>>>     should
>>>     > design
>>>     > >> the return type of the oneToManyJoin and how many power we
>>>     want to give
>>>     > to
>>>     > >> the user vs "simplicity" (where simplicity isn't really that
>>>     as users
>>>     > still
>>>     > >> need to understand it I argue)
>>>     > >>
>>>     > >> waiting for you to join in on the discussion
>>>     > >>
>>>     > >> Best Jan
>>>     > >>
>>>     > >>
>>>     > >>
>>>     > >> On 07.09.2018 15:49, James Kwan wrote:
>>>     > >>
>>>     > >>> I am new to this group and I found this subject
>>>     interesting.  Sounds
>>>     > like
>>>     > >>> you guys want to implement a join table of two streams? Is
>>> there
>>>     > somewhere
>>>     > >>> I can see the original requirement or proposal?
>>>     > >>>
>>>     > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>
>>>     > >>>> wrote:
>>>     > >>>>
>>>     > >>>>
>>>     > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>     > >>>>
>>>     > >>>>> I'm currently testing using a Windowed Store to store the
>>>     highwater
>>>     > >>>>> mark.
>>>     > >>>>> By all indications this should work fine, with the caveat
>>>     being that
>>>     > it
>>>     > >>>>> can
>>>     > >>>>> only resolve out-of-order arrival for up to the size of
>>>     the window
>>>     > (ie:
>>>     > >>>>> 24h, 72h, etc). This would remove the possibility of it being
>>>     > unbounded
>>>     > >>>>> in
>>>     > >>>>> size.
>>>     > >>>>>
>>>     > >>>>> With regards to Jan's suggestion, I believe this is where
>>>     we will
>>>     > have
>>>     > >>>>> to
>>>     > >>>>> remain in disagreement. While I do not disagree with your
>>>     statement
>>>     > >>>>> about
>>>     > >>>>> there likely to be additional joins done in a real-world
>>>     workflow, I
>>>     > do
>>>     > >>>>> not
>>>     > >>>>> see how you can conclusively deal with out-of-order arrival
>>> of
>>>     > >>>>> foreign-key
>>>     > >>>>> changes and subsequent joins. I have attempted what I
>>>     think you have
>>>     > >>>>> proposed (without a high-water, using groupBy and reduce)
>>>     and found
>>>     > >>>>> that if
>>>     > >>>>> the foreign key changes too quickly, or the load on a
>>>     stream thread
>>>     > is
>>>     > >>>>> too
>>>     > >>>>> high, the joined messages will arrive out-of-order and be
>>>     incorrectly
>>>     > >>>>> propagated, such that an intermediate event is represented
>>>     as the
>>>     > final
>>>     > >>>>> event.
>>>     > >>>>>
>>>     > >>>> Can you shed some light on your groupBy implementation.
>>>     There must be
>>>     > >>>> some sort of flaw in it.
>>>     > >>>> I have a suspicion where it is, I would just like to
>>>     confirm. The idea
>>>     > >>>> is bullet proof and it must be
>>>     > >>>> an implementation mess up. I would like to clarify before
>>>     we draw a
>>>     > >>>> conclusion.
>>>     > >>>>
>>>     > >>>>    Repartitioning the scattered events back to their original
>>>     > >>>>> partitions is the only way I know how to conclusively deal
>>>     with
>>>     > >>>>> out-of-order events in a given time frame, and to ensure
>>>     that the
>>>     > data
>>>     > >>>>> is
>>>     > >>>>> eventually consistent with the input events.
>>>     > >>>>>
>>>     > >>>>> If you have some code to share that illustrates your
>>>     approach, I
>>>     > would
>>>     > >>>>> be
>>>     > >>>>> very grateful as it would remove any misunderstandings
>>>     that I may
>>>     > have.
>>>     > >>>>>
>>>     > >>>> ah okay you were looking for my code. I don't have
>>>     something easily
>>>     > >>>> readable here as its bloated with OO-patterns.
>>>     > >>>>
>>>     > >>>> its anyhow trivial:
>>>     > >>>>
>>>     > >>>> @Override
>>>     > >>>>      public T apply(K aggKey, V value, T aggregate)
>>>     > >>>>      {
>>>     > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
>>>     imaginary
>>>     > >>>>          U toModifyKey = mapper.apply(value);
>>>     > >>>>              << this is the place where people actually
>>>     gonna have
>>>     > issues
>>>     > >>>> and why you probably couldn't do it. we would need to find
>>>     a solution
>>>     > here.
>>>     > >>>> I didn't realize that yet.
>>>     > >>>>              << we propagate the field in the joiner, so
>>>     that we can
>>>     > pick
>>>     > >>>> it up in an aggregate. Probably you have not thought of
>>>     this in your
>>>     > >>>> approach right?
>>>     > >>>>              << I am very open to find a generic solution
>>>     here. In my
>>>     > >>>> honest opinion this is broken in KTableImpl.GroupBy that it
>>>     looses
>>>     > the keys
>>>     > >>>> and only maintains the aggregate key.
>>>     > >>>>              << I abstracted it away back then way before i
>>> was
>>>     > thinking
>>>     > >>>> of oneToMany join. That is why I didn't realize its
>>>     significance here.
>>>     > >>>>              << Opinions?
>>>     > >>>>
>>>     > >>>>          for (V m : current)
>>>     > >>>>          {
>>>     > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>>     > >>>>          }
>>>     > >>>>          if (isAdder)
>>>     > >>>>          {
>>>     > >>>> currentStateAsMap.put(toModifyKey, value);
>>>     > >>>>          }
>>>     > >>>>          else
>>>     > >>>>          {
>>>     > >>>> currentStateAsMap.remove(toModifyKey);
>>>     > >>>> if(currentStateAsMap.isEmpty()){
>>>     > >>>>                  return null;
>>>     > >>>>              }
>>>     > >>>>          }
>>>     > >>>>          retrun asAggregateType(currentStateAsMap)
>>>     > >>>>      }
>>>     > >>>>
>>>     > >>>>
>>>     > >>>>
>>>     > >>>>
>>>     > >>>>
>>>     > >>>> Thanks,
>>>     > >>>>> Adam
>>>     > >>>>>
>>>     > >>>>>
>>>     > >>>>>
>>>     > >>>>>
>>>     > >>>>>
>>>     > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>>     > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>>
>>>     > >>>>> wrote:
>>>     > >>>>>
>>>     > >>>>> Thanks Adam for bringing Matthias to speed!
>>>     > >>>>>> about the differences. I think re-keying back should be
>>>     optional at
>>>     > >>>>>> best.
>>>     > >>>>>> I would say we return a KScatteredTable with reshuffle()
>>>     returning
>>>     > >>>>>> KTable<originalKey,Joined> to make the backwards
>>>     repartitioning
>>>     > >>>>>> optional.
>>>     > >>>>>> I am also in a big favour of doing the out of order
>>>     processing using
>>>     > >>>>>> group
>>>     > >>>>>> by instead high water mark tracking.
>>>     > >>>>>> Just because unbounded growth is just scary + It saves us
>>>     the header
>>>     > >>>>>> stuff.
>>>     > >>>>>>
>>>     > >>>>>> I think the abstraction of always repartitioning back is
>>>     just not so
>>>     > >>>>>> strong. Like the work has been done before we partition
>>>     back and
>>>     > >>>>>> grouping
>>>     > >>>>>> by something else afterwards is really common.
>>>     > >>>>>>
>>>     > >>>>>>
>>>     > >>>>>>
>>>     > >>>>>>
>>>     > >>>>>>
>>>     > >>>>>>
>>>     > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>     > >>>>>>
>>>     > >>>>>> Hi Matthias
>>>     > >>>>>>> Thank you for your feedback, I do appreciate it!
>>>     > >>>>>>>
>>>     > >>>>>>> While name spacing would be possible, it would require to
>>>     > deserialize
>>>     > >>>>>>>
>>>     > >>>>>>>> user headers what implies a runtime overhead. I would
>>>     suggest to
>>>     > no
>>>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>>>     > problem in
>>>     > >>>>>>>> the future, we can still add name spacing later on.
>>>     > >>>>>>>>
>>>     > >>>>>>>> Agreed. I will go with using a reserved string and
>>>     document it.
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>> My main concern about the design it the type of the
>>>     result KTable:
>>>     > If
>>>     > >>>>>>> I
>>>     > >>>>>>> understood the proposal correctly,
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>> In your example, you have table1 and table2 swapped.
>>>     Here is how it
>>>     > >>>>>>> works
>>>     > >>>>>>> currently:
>>>     > >>>>>>>
>>>     > >>>>>>> 1) table1 has the records that contain the foreign key
>>>     within their
>>>     > >>>>>>> value.
>>>     > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>>>     > >>>>>>> <c,(fk=B,bar=3)>
>>>     > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>>     > >>>>>>>
>>>     > >>>>>>> 2) A Value mapper is required to extract the foreign key.
>>>     > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>>     <http://value.fk> )
>>>
>>>     > >>>>>>>
>>>     > >>>>>>> The mapper is applied to each element in table1, and a
>>>     new combined
>>>     > >>>>>>> key is
>>>     > >>>>>>> made:
>>>     > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>,
>>>     <B-c,
>>>     > >>>>>>> (fk=B,bar=3)>
>>>     > >>>>>>>
>>>     > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>     > >>>>>>>
>>>     > >>>>>>> a) Stream Thread with Partition 0:
>>>     > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>>     (fk=A,bar=2)>
>>>     > >>>>>>> Table2: <A,X>
>>>     > >>>>>>>
>>>     > >>>>>>> b) Stream Thread with Partition 1:
>>>     > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>     > >>>>>>> Table2: <B,Y>
>>>     > >>>>>>>
>>>     > >>>>>>> 4) From here, they can be joined together locally by
>>>     applying the
>>>     > >>>>>>> joiner
>>>     > >>>>>>> function.
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>> At this point, Jan's design and my design deviate. My
>>>     design goes
>>>     > on
>>>     > >>>>>>> to
>>>     > >>>>>>> repartition the data post-join and resolve out-of-order
>>>     arrival of
>>>     > >>>>>>> records,
>>>     > >>>>>>> finally returning the data keyed just the original key.
>>>     I do not
>>>     > >>>>>>> expose
>>>     > >>>>>>> the
>>>     > >>>>>>> CombinedKey or any of the internals outside of the
>>>     joinOnForeignKey
>>>     > >>>>>>> function. This does make for larger footprint, but it
>>>     removes all
>>>     > >>>>>>> agency
>>>     > >>>>>>> for resolving out-of-order arrivals and handling
>>>     CombinedKeys from
>>>     > the
>>>     > >>>>>>> user. I believe that this makes the function much easier
>>>     to use.
>>>     > >>>>>>>
>>>     > >>>>>>> Let me know if this helps resolve your questions, and
>>>     please feel
>>>     > >>>>>>> free to
>>>     > >>>>>>> add anything else on your mind.
>>>     > >>>>>>>
>>>     > >>>>>>> Thanks again,
>>>     > >>>>>>> Adam
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>>
>>>     > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>     > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>>>
>>>     > >>>>>>> wrote:
>>>     > >>>>>>>
>>>     > >>>>>>> Hi,
>>>     > >>>>>>>
>>>     > >>>>>>>> I am just catching up on this thread. I did not read
>>>     everything so
>>>     > >>>>>>>> far,
>>>     > >>>>>>>> but want to share couple of initial thoughts:
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>> Headers: I think there is a fundamental difference
>>>     between header
>>>     > >>>>>>>> usage
>>>     > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>>>     changelog topic
>>>     > >>>>>>>> that
>>>     > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>>>     to write
>>>     > into
>>>     > >>>>>>>> them. In fact, no user header are written into the
>>>     changelog topic
>>>     > >>>>>>>> and
>>>     > >>>>>>>> thus, there are not conflicts.
>>>     > >>>>>>>>
>>>     > >>>>>>>> Nevertheless, I don't see a big issue with using
>>>     headers within
>>>     > >>>>>>>> Streams.
>>>     > >>>>>>>> As long as we document it, we can have some "reserved"
>>>     header keys
>>>     > >>>>>>>> and
>>>     > >>>>>>>> users are not allowed to use when processing data with
>>>     Kafka
>>>     > Streams.
>>>     > >>>>>>>> IMHO, this should be ok.
>>>     > >>>>>>>>
>>>     > >>>>>>>> I think there is a safe way to avoid conflicts, since
>>> these
>>>     > headers
>>>     > >>>>>>>> are
>>>     > >>>>>>>>
>>>     > >>>>>>>>> only needed in internal topics (I think):
>>>     > >>>>>>>>> For internal and changelog topics, we can namespace
>>>     all headers:
>>>     > >>>>>>>>> * user-defined headers are namespaced as "external." +
>>>     headerKey
>>>     > >>>>>>>>> * internal headers are namespaced as "internal." +
>>>     headerKey
>>>     > >>>>>>>>>
>>>     > >>>>>>>>> While name spacing would be possible, it would require to
>>>     > >>>>>>>> deserialize
>>>     > >>>>>>>> user headers what implies a runtime overhead. I would
>>>     suggest to
>>>     > no
>>>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>>>     > problem in
>>>     > >>>>>>>> the future, we can still add name spacing later on.
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>> My main concern about the design it the type of the
>>>     result KTable:
>>>     > >>>>>>>> If I
>>>     > >>>>>>>> understood the proposal correctly,
>>>     > >>>>>>>>
>>>     > >>>>>>>> KTable<K1,V1> table1 = ...
>>>     > >>>>>>>> KTable<K2,V2> table2 = ...
>>>     > >>>>>>>>
>>>     > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>     > >>>>>>>>
>>>     > >>>>>>>> implies that the `joinedTable` has the same key as the
>>>     left input
>>>     > >>>>>>>> table.
>>>     > >>>>>>>> IMHO, this does not work because if table2 contains
>>>     multiple rows
>>>     > >>>>>>>> that
>>>     > >>>>>>>> join with a record in table1 (what is the main purpose of
>>> a
>>>     > foreign
>>>     > >>>>>>>> key
>>>     > >>>>>>>> join), the result table would only contain a single
>>>     join result,
>>>     > but
>>>     > >>>>>>>> not
>>>     > >>>>>>>> multiple.
>>>     > >>>>>>>>
>>>     > >>>>>>>> Example:
>>>     > >>>>>>>>
>>>     > >>>>>>>> table1 input stream: <A,X>
>>>     > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>     > >>>>>>>>
>>>     > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>>>     "A" joins).
>>>     > If
>>>     > >>>>>>>> the
>>>     > >>>>>>>> result key is the same key as key of table1, this
>>>     implies that the
>>>     > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>>>     but not
>>>     > both.
>>>     > >>>>>>>> Because the share the same key, whatever result record
>>>     we emit
>>>     > later,
>>>     > >>>>>>>> overwrite the previous result.
>>>     > >>>>>>>>
>>>     > >>>>>>>> This is the reason why Jan originally proposed to use a
>>>     > combination
>>>     > >>>>>>>> of
>>>     > >>>>>>>> both primary keys of the input tables as key of the
>>>     output table.
>>>     > >>>>>>>> This
>>>     > >>>>>>>> makes the keys of the output table unique and we can
>>>     store both in
>>>     > >>>>>>>> the
>>>     > >>>>>>>> output table:
>>>     > >>>>>>>>
>>>     > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>> Thoughts?
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>> -Matthias
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>>
>>>     > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>     > >>>>>>>>
>>>     > >>>>>>>> Just on remark here.
>>>     > >>>>>>>>> The high-watermark could be disregarded. The decision
>>>     about the
>>>     > >>>>>>>>> forward
>>>     > >>>>>>>>> depends on the size of the aggregated map.
>>>     > >>>>>>>>> Only 1 element long maps would be unpacked and
>>>     forwarded. 0
>>>     > element
>>>     > >>>>>>>>> maps
>>>     > >>>>>>>>> would be published as delete. Any other count
>>>     > >>>>>>>>> of map entries is in "waiting for correct deletes to
>>>     > arrive"-state.
>>>     > >>>>>>>>>
>>>     > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>     > >>>>>>>>>
>>>     > >>>>>>>>> It does look like I could replace the second
>>>     repartition store
>>>     > and
>>>     > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>>>     it looks
>>>     > like
>>>     > >>>>>>>>>> I
>>>     > >>>>>>>>>> would
>>>     > >>>>>>>>>> still need to store the highwater value within the
>>>     materialized
>>>     > >>>>>>>>>> store,
>>>     > >>>>>>>>>>
>>>     > >>>>>>>>>> to
>>>     > >>>>>>>>> compare the arrival of out-of-order records (assuming my
>>>     > >>>>>>>>> understanding
>>>     > >>>>>>>>> of
>>>     > >>>>>>>>> THIS is correct...). This in effect is the same as the
>>>     design I
>>>     > have
>>>     > >>>>>>>>> now,
>>>     > >>>>>>>>> just with the two tables merged together.
>>>     > >>>>>>>>>
>>>     >
>>>     >
>>>
>>>
>>>
>>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

Thanks for the explanation. Regarding the final step (i.e. the high
watermark store, now altered to be replaced with a window store), I think
another current on-going KIP may actually help:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB


This is for adding the timestamp into a key-value store (i.e. only for
non-windowed KTable), and then one of its usage, as described in
https://issues.apache.org/jira/browse/KAFKA-5533, is that we can then
"reject" updates from the source topics if its timestamp is smaller than
the current key's latest update timestamp. I think it is very similar to
what you have in mind for high watermark based filtering, while you only
need to make sure that the timestamps of the joining records are correctly
inherited though the whole topology to the final stage.

Note that this KIP is for key-value store and hence non-windowed KTables
only, but for windowed KTables we do not really have a good support for
their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107) I
think we can just consider non-windowed KTable-KTable non-key joins for
now. In which case, KIP-258 should help.



Guozhang



On Wed, Sep 12, 2018 at 9:20 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

>
> On 11.09.2018 18:00, Adam Bellemare wrote:
>
>> Hi Guozhang
>>
>> Current highwater mark implementation would grow endlessly based on
>> primary key of original event. It is a pair of (<this table primary key>,
>> <highest offset seen for that key>). This is used to differentiate between
>> late arrivals and new updates. My newest proposal would be to replace it
>> with a Windowed state store of Duration N. This would allow the same
>> behaviour, but cap the size based on time. This should allow for all
>> late-arriving events to be processed, and should be customizable by the
>> user to tailor to their own needs (ie: perhaps just 10 minutes of window,
>> or perhaps 7 days...).
>>
> Hi Adam, using time based retention can do the trick here. Even if I would
> still like to see the automatic repartitioning optional since I would just
> reshuffle again. With windowed store I am a little bit sceptical about how
> to determine the window. So esentially one could run into problems when the
> rapid change happens near a window border. I will check you implementation
> in detail, if its problematic, we could still check _all_ windows on read
> with not to bad performance impact I guess. Will let you know if the
> implementation would be correct as is. I wouldn't not like to assume that:
> offset(A) < offset(B) => timestamp(A)  < timestamp(B). I think we can't
> expect that.
>
>>
>>
>> @Jan
>> I believe I understand what you mean now - thanks for the diagram, it did
>> really help. You are correct that I do not have the original primary key
>> available, and I can see that if it was available then you would be able to
>> add and remove events from the Map. That being said, I encourage you to
>> finish your diagrams / charts just for clarity for everyone else.
>>
>> Yeah 100%, this giphy thing is just really hard work. But I understand
> the benefits for the rest. Sorry about the original primary key, We have
> join and Group by implemented our own in PAPI and basically not using any
> DSL (Just the abstraction). Completely missed that in original DSL its not
> there and just assumed it. total brain mess up on my end. Will finish the
> chart as soon as i get a quite evening this week.
>
> My follow up question for you is, won't the Map stay inside the State
>> Store indefinitely after all of the changes have propagated? Isn't this
>> effectively the same as a highwater mark state store?
>>
> Thing is that if the map is empty, substractor is gonna return `null` and
> the key is removed from the keyspace. But there is going to be a store
> 100%, the good thing is that I can use this store directly for
> materialize() / enableSendingOldValues() is a regular store, satisfying
> all gurantees needed for further groupby / join. The Windowed store is not
> keeping the values, so for the next statefull operation we would
> need to instantiate an extra store. or we have the window store also have
> the values then.
>
> Long story short. if we can flip in a custom group by before
> repartitioning to the original primary key i think it would help the users
> big time in building efficient apps. Given the original primary key issue I
> understand that we do not have a solid foundation to build on.
> Leaving primary key carry along to the user. very unfortunate. I could
> understand the decision goes like that. I do not think its a good decision.
>
>
>>
>>
>>
>> Thanks
>> Adam
>>
>>
>>
>>
>>
>>
>> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
>> dumbreprajakta311@gmail.com <ma...@gmail.com>> wrote:
>>
>>     please remove me from this group
>>
>>     On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>
>>     wrote:
>>
>>     > Hi Adam,
>>     >
>>     > give me some time, will make such a chart. last time i didn't
>>     get along
>>     > well with giphy and ruined all your charts.
>>     > Hopefully i can get it done today
>>     >
>>     > On 08.09.2018 16:00, Adam Bellemare wrote:
>>     > > Hi Jan
>>     > >
>>     > > I have included a diagram of what I attempted on the KIP.
>>     > >
>>     >
>>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjo
>> ininginKTable-GroupBy+Reduce/Aggregate
>>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjo
>> ininginKTable-GroupBy+Reduce/Aggregate>
>>     > >
>>     > > I attempted this back at the start of my own implementation of
>>     this
>>     > > solution, and since I could not get it to work I have since
>>     discarded the
>>     > > code. At this point in time, if you wish to continue pursuing
>>     for your
>>     > > groupBy solution, I ask that you please create a diagram on
>>     the KIP
>>     > > carefully explaining your solution. Please feel free to use
>>     the image I
>>     > > just posted as a starting point. I am having trouble
>>     understanding your
>>     > > explanations but I think that a carefully constructed diagram
>>     will clear
>>     > up
>>     > > any misunderstandings. Alternately, please post a
>>     comprehensive PR with
>>     > > your solution. I can only guess at what you mean, and since I
>>     value my
>>     > own
>>     > > time as much as you value yours, I believe it is your
>>     responsibility to
>>     > > provide an implementation instead of me trying to guess.
>>     > >
>>     > > Adam
>>     > >
>>     > >
>>     > >
>>     > >
>>     > >
>>     > >
>>     > >
>>     > >
>>     > >
>>     > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>
>>     > > wrote:
>>     > >
>>     > >> Hi James,
>>     > >>
>>     > >> nice to see you beeing interested. kafka streams at this
>>     point supports
>>     > >> all sorts of joins as long as both streams have the same key.
>>     > >> Adam is currently implementing a join where a KTable and a
>>     KTable can
>>     > have
>>     > >> a one to many relation ship (1:n). We exploit that rocksdb is a
>>     > >> datastore that keeps data sorted (At least exposes an API to
>>     access the
>>     > >> stored data in a sorted fashion).
>>     > >>
>>     > >> I think the technical caveats are well understood now and we are
>>     > basically
>>     > >> down to philosophy and API Design ( when Adam sees my newest
>>     message).
>>     > >> I have a lengthy track record of loosing those kinda
>>     arguments within
>>     > the
>>     > >> streams community and I have no clue why. So I literally
>>     can't wait for
>>     > you
>>     > >> to churn through this thread and give you opinion on how we
>>     should
>>     > design
>>     > >> the return type of the oneToManyJoin and how many power we
>>     want to give
>>     > to
>>     > >> the user vs "simplicity" (where simplicity isn't really that
>>     as users
>>     > still
>>     > >> need to understand it I argue)
>>     > >>
>>     > >> waiting for you to join in on the discussion
>>     > >>
>>     > >> Best Jan
>>     > >>
>>     > >>
>>     > >>
>>     > >> On 07.09.2018 15:49, James Kwan wrote:
>>     > >>
>>     > >>> I am new to this group and I found this subject
>>     interesting.  Sounds
>>     > like
>>     > >>> you guys want to implement a join table of two streams? Is there
>>     > somewhere
>>     > >>> I can see the original requirement or proposal?
>>     > >>>
>>     > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>
>>     > >>>> wrote:
>>     > >>>>
>>     > >>>>
>>     > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>     > >>>>
>>     > >>>>> I'm currently testing using a Windowed Store to store the
>>     highwater
>>     > >>>>> mark.
>>     > >>>>> By all indications this should work fine, with the caveat
>>     being that
>>     > it
>>     > >>>>> can
>>     > >>>>> only resolve out-of-order arrival for up to the size of
>>     the window
>>     > (ie:
>>     > >>>>> 24h, 72h, etc). This would remove the possibility of it being
>>     > unbounded
>>     > >>>>> in
>>     > >>>>> size.
>>     > >>>>>
>>     > >>>>> With regards to Jan's suggestion, I believe this is where
>>     we will
>>     > have
>>     > >>>>> to
>>     > >>>>> remain in disagreement. While I do not disagree with your
>>     statement
>>     > >>>>> about
>>     > >>>>> there likely to be additional joins done in a real-world
>>     workflow, I
>>     > do
>>     > >>>>> not
>>     > >>>>> see how you can conclusively deal with out-of-order arrival of
>>     > >>>>> foreign-key
>>     > >>>>> changes and subsequent joins. I have attempted what I
>>     think you have
>>     > >>>>> proposed (without a high-water, using groupBy and reduce)
>>     and found
>>     > >>>>> that if
>>     > >>>>> the foreign key changes too quickly, or the load on a
>>     stream thread
>>     > is
>>     > >>>>> too
>>     > >>>>> high, the joined messages will arrive out-of-order and be
>>     incorrectly
>>     > >>>>> propagated, such that an intermediate event is represented
>>     as the
>>     > final
>>     > >>>>> event.
>>     > >>>>>
>>     > >>>> Can you shed some light on your groupBy implementation.
>>     There must be
>>     > >>>> some sort of flaw in it.
>>     > >>>> I have a suspicion where it is, I would just like to
>>     confirm. The idea
>>     > >>>> is bullet proof and it must be
>>     > >>>> an implementation mess up. I would like to clarify before
>>     we draw a
>>     > >>>> conclusion.
>>     > >>>>
>>     > >>>>    Repartitioning the scattered events back to their original
>>     > >>>>> partitions is the only way I know how to conclusively deal
>>     with
>>     > >>>>> out-of-order events in a given time frame, and to ensure
>>     that the
>>     > data
>>     > >>>>> is
>>     > >>>>> eventually consistent with the input events.
>>     > >>>>>
>>     > >>>>> If you have some code to share that illustrates your
>>     approach, I
>>     > would
>>     > >>>>> be
>>     > >>>>> very grateful as it would remove any misunderstandings
>>     that I may
>>     > have.
>>     > >>>>>
>>     > >>>> ah okay you were looking for my code. I don't have
>>     something easily
>>     > >>>> readable here as its bloated with OO-patterns.
>>     > >>>>
>>     > >>>> its anyhow trivial:
>>     > >>>>
>>     > >>>> @Override
>>     > >>>>      public T apply(K aggKey, V value, T aggregate)
>>     > >>>>      {
>>     > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
>>     imaginary
>>     > >>>>          U toModifyKey = mapper.apply(value);
>>     > >>>>              << this is the place where people actually
>>     gonna have
>>     > issues
>>     > >>>> and why you probably couldn't do it. we would need to find
>>     a solution
>>     > here.
>>     > >>>> I didn't realize that yet.
>>     > >>>>              << we propagate the field in the joiner, so
>>     that we can
>>     > pick
>>     > >>>> it up in an aggregate. Probably you have not thought of
>>     this in your
>>     > >>>> approach right?
>>     > >>>>              << I am very open to find a generic solution
>>     here. In my
>>     > >>>> honest opinion this is broken in KTableImpl.GroupBy that it
>>     looses
>>     > the keys
>>     > >>>> and only maintains the aggregate key.
>>     > >>>>              << I abstracted it away back then way before i was
>>     > thinking
>>     > >>>> of oneToMany join. That is why I didn't realize its
>>     significance here.
>>     > >>>>              << Opinions?
>>     > >>>>
>>     > >>>>          for (V m : current)
>>     > >>>>          {
>>     > >>>> currentStateAsMap.put(mapper.apply(m), m);
>>     > >>>>          }
>>     > >>>>          if (isAdder)
>>     > >>>>          {
>>     > >>>> currentStateAsMap.put(toModifyKey, value);
>>     > >>>>          }
>>     > >>>>          else
>>     > >>>>          {
>>     > >>>> currentStateAsMap.remove(toModifyKey);
>>     > >>>> if(currentStateAsMap.isEmpty()){
>>     > >>>>                  return null;
>>     > >>>>              }
>>     > >>>>          }
>>     > >>>>          retrun asAggregateType(currentStateAsMap)
>>     > >>>>      }
>>     > >>>>
>>     > >>>>
>>     > >>>>
>>     > >>>>
>>     > >>>>
>>     > >>>> Thanks,
>>     > >>>>> Adam
>>     > >>>>>
>>     > >>>>>
>>     > >>>>>
>>     > >>>>>
>>     > >>>>>
>>     > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>>     > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>>
>>     > >>>>> wrote:
>>     > >>>>>
>>     > >>>>> Thanks Adam for bringing Matthias to speed!
>>     > >>>>>> about the differences. I think re-keying back should be
>>     optional at
>>     > >>>>>> best.
>>     > >>>>>> I would say we return a KScatteredTable with reshuffle()
>>     returning
>>     > >>>>>> KTable<originalKey,Joined> to make the backwards
>>     repartitioning
>>     > >>>>>> optional.
>>     > >>>>>> I am also in a big favour of doing the out of order
>>     processing using
>>     > >>>>>> group
>>     > >>>>>> by instead high water mark tracking.
>>     > >>>>>> Just because unbounded growth is just scary + It saves us
>>     the header
>>     > >>>>>> stuff.
>>     > >>>>>>
>>     > >>>>>> I think the abstraction of always repartitioning back is
>>     just not so
>>     > >>>>>> strong. Like the work has been done before we partition
>>     back and
>>     > >>>>>> grouping
>>     > >>>>>> by something else afterwards is really common.
>>     > >>>>>>
>>     > >>>>>>
>>     > >>>>>>
>>     > >>>>>>
>>     > >>>>>>
>>     > >>>>>>
>>     > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>     > >>>>>>
>>     > >>>>>> Hi Matthias
>>     > >>>>>>> Thank you for your feedback, I do appreciate it!
>>     > >>>>>>>
>>     > >>>>>>> While name spacing would be possible, it would require to
>>     > deserialize
>>     > >>>>>>>
>>     > >>>>>>>> user headers what implies a runtime overhead. I would
>>     suggest to
>>     > no
>>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>>     > problem in
>>     > >>>>>>>> the future, we can still add name spacing later on.
>>     > >>>>>>>>
>>     > >>>>>>>> Agreed. I will go with using a reserved string and
>>     document it.
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>> My main concern about the design it the type of the
>>     result KTable:
>>     > If
>>     > >>>>>>> I
>>     > >>>>>>> understood the proposal correctly,
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>> In your example, you have table1 and table2 swapped.
>>     Here is how it
>>     > >>>>>>> works
>>     > >>>>>>> currently:
>>     > >>>>>>>
>>     > >>>>>>> 1) table1 has the records that contain the foreign key
>>     within their
>>     > >>>>>>> value.
>>     > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>>     > >>>>>>> <c,(fk=B,bar=3)>
>>     > >>>>>>> table2 input stream: <A,X>, <B,Y>
>>     > >>>>>>>
>>     > >>>>>>> 2) A Value mapper is required to extract the foreign key.
>>     > >>>>>>> table1 foreign key mapper: ( value => value.fk
>>     <http://value.fk> )
>>
>>     > >>>>>>>
>>     > >>>>>>> The mapper is applied to each element in table1, and a
>>     new combined
>>     > >>>>>>> key is
>>     > >>>>>>> made:
>>     > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>,
>>     <B-c,
>>     > >>>>>>> (fk=B,bar=3)>
>>     > >>>>>>>
>>     > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>>     > >>>>>>>
>>     > >>>>>>> a) Stream Thread with Partition 0:
>>     > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>>     (fk=A,bar=2)>
>>     > >>>>>>> Table2: <A,X>
>>     > >>>>>>>
>>     > >>>>>>> b) Stream Thread with Partition 1:
>>     > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>     > >>>>>>> Table2: <B,Y>
>>     > >>>>>>>
>>     > >>>>>>> 4) From here, they can be joined together locally by
>>     applying the
>>     > >>>>>>> joiner
>>     > >>>>>>> function.
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>> At this point, Jan's design and my design deviate. My
>>     design goes
>>     > on
>>     > >>>>>>> to
>>     > >>>>>>> repartition the data post-join and resolve out-of-order
>>     arrival of
>>     > >>>>>>> records,
>>     > >>>>>>> finally returning the data keyed just the original key.
>>     I do not
>>     > >>>>>>> expose
>>     > >>>>>>> the
>>     > >>>>>>> CombinedKey or any of the internals outside of the
>>     joinOnForeignKey
>>     > >>>>>>> function. This does make for larger footprint, but it
>>     removes all
>>     > >>>>>>> agency
>>     > >>>>>>> for resolving out-of-order arrivals and handling
>>     CombinedKeys from
>>     > the
>>     > >>>>>>> user. I believe that this makes the function much easier
>>     to use.
>>     > >>>>>>>
>>     > >>>>>>> Let me know if this helps resolve your questions, and
>>     please feel
>>     > >>>>>>> free to
>>     > >>>>>>> add anything else on your mind.
>>     > >>>>>>>
>>     > >>>>>>> Thanks again,
>>     > >>>>>>> Adam
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>>
>>     > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>     > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>>
>>     > >>>>>>> wrote:
>>     > >>>>>>>
>>     > >>>>>>> Hi,
>>     > >>>>>>>
>>     > >>>>>>>> I am just catching up on this thread. I did not read
>>     everything so
>>     > >>>>>>>> far,
>>     > >>>>>>>> but want to share couple of initial thoughts:
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>> Headers: I think there is a fundamental difference
>>     between header
>>     > >>>>>>>> usage
>>     > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>>     changelog topic
>>     > >>>>>>>> that
>>     > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>>     to write
>>     > into
>>     > >>>>>>>> them. In fact, no user header are written into the
>>     changelog topic
>>     > >>>>>>>> and
>>     > >>>>>>>> thus, there are not conflicts.
>>     > >>>>>>>>
>>     > >>>>>>>> Nevertheless, I don't see a big issue with using
>>     headers within
>>     > >>>>>>>> Streams.
>>     > >>>>>>>> As long as we document it, we can have some "reserved"
>>     header keys
>>     > >>>>>>>> and
>>     > >>>>>>>> users are not allowed to use when processing data with
>>     Kafka
>>     > Streams.
>>     > >>>>>>>> IMHO, this should be ok.
>>     > >>>>>>>>
>>     > >>>>>>>> I think there is a safe way to avoid conflicts, since these
>>     > headers
>>     > >>>>>>>> are
>>     > >>>>>>>>
>>     > >>>>>>>>> only needed in internal topics (I think):
>>     > >>>>>>>>> For internal and changelog topics, we can namespace
>>     all headers:
>>     > >>>>>>>>> * user-defined headers are namespaced as "external." +
>>     headerKey
>>     > >>>>>>>>> * internal headers are namespaced as "internal." +
>>     headerKey
>>     > >>>>>>>>>
>>     > >>>>>>>>> While name spacing would be possible, it would require to
>>     > >>>>>>>> deserialize
>>     > >>>>>>>> user headers what implies a runtime overhead. I would
>>     suggest to
>>     > no
>>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>>     > problem in
>>     > >>>>>>>> the future, we can still add name spacing later on.
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>> My main concern about the design it the type of the
>>     result KTable:
>>     > >>>>>>>> If I
>>     > >>>>>>>> understood the proposal correctly,
>>     > >>>>>>>>
>>     > >>>>>>>> KTable<K1,V1> table1 = ...
>>     > >>>>>>>> KTable<K2,V2> table2 = ...
>>     > >>>>>>>>
>>     > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>     > >>>>>>>>
>>     > >>>>>>>> implies that the `joinedTable` has the same key as the
>>     left input
>>     > >>>>>>>> table.
>>     > >>>>>>>> IMHO, this does not work because if table2 contains
>>     multiple rows
>>     > >>>>>>>> that
>>     > >>>>>>>> join with a record in table1 (what is the main purpose of a
>>     > foreign
>>     > >>>>>>>> key
>>     > >>>>>>>> join), the result table would only contain a single
>>     join result,
>>     > but
>>     > >>>>>>>> not
>>     > >>>>>>>> multiple.
>>     > >>>>>>>>
>>     > >>>>>>>> Example:
>>     > >>>>>>>>
>>     > >>>>>>>> table1 input stream: <A,X>
>>     > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>     > >>>>>>>>
>>     > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>>     "A" joins).
>>     > If
>>     > >>>>>>>> the
>>     > >>>>>>>> result key is the same key as key of table1, this
>>     implies that the
>>     > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>>     but not
>>     > both.
>>     > >>>>>>>> Because the share the same key, whatever result record
>>     we emit
>>     > later,
>>     > >>>>>>>> overwrite the previous result.
>>     > >>>>>>>>
>>     > >>>>>>>> This is the reason why Jan originally proposed to use a
>>     > combination
>>     > >>>>>>>> of
>>     > >>>>>>>> both primary keys of the input tables as key of the
>>     output table.
>>     > >>>>>>>> This
>>     > >>>>>>>> makes the keys of the output table unique and we can
>>     store both in
>>     > >>>>>>>> the
>>     > >>>>>>>> output table:
>>     > >>>>>>>>
>>     > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>> Thoughts?
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>> -Matthias
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>>
>>     > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>     > >>>>>>>>
>>     > >>>>>>>> Just on remark here.
>>     > >>>>>>>>> The high-watermark could be disregarded. The decision
>>     about the
>>     > >>>>>>>>> forward
>>     > >>>>>>>>> depends on the size of the aggregated map.
>>     > >>>>>>>>> Only 1 element long maps would be unpacked and
>>     forwarded. 0
>>     > element
>>     > >>>>>>>>> maps
>>     > >>>>>>>>> would be published as delete. Any other count
>>     > >>>>>>>>> of map entries is in "waiting for correct deletes to
>>     > arrive"-state.
>>     > >>>>>>>>>
>>     > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>     > >>>>>>>>>
>>     > >>>>>>>>> It does look like I could replace the second
>>     repartition store
>>     > and
>>     > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>>     it looks
>>     > like
>>     > >>>>>>>>>> I
>>     > >>>>>>>>>> would
>>     > >>>>>>>>>> still need to store the highwater value within the
>>     materialized
>>     > >>>>>>>>>> store,
>>     > >>>>>>>>>>
>>     > >>>>>>>>>> to
>>     > >>>>>>>>> compare the arrival of out-of-order records (assuming my
>>     > >>>>>>>>> understanding
>>     > >>>>>>>>> of
>>     > >>>>>>>>> THIS is correct...). This in effect is the same as the
>>     design I
>>     > have
>>     > >>>>>>>>> now,
>>     > >>>>>>>>> just with the two tables merged together.
>>     > >>>>>>>>>
>>     >
>>     >
>>
>>
>>
>


-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
On 11.09.2018 18:00, Adam Bellemare wrote:
> Hi Guozhang
>
> Current highwater mark implementation would grow endlessly based on 
> primary key of original event. It is a pair of (<this table primary 
> key>, <highest offset seen for that key>). This is used to 
> differentiate between late arrivals and new updates. My newest 
> proposal would be to replace it with a Windowed state store of 
> Duration N. This would allow the same behaviour, but cap the size 
> based on time. This should allow for all late-arriving events to be 
> processed, and should be customizable by the user to tailor to their 
> own needs (ie: perhaps just 10 minutes of window, or perhaps 7 days...).
Hi Adam, using time based retention can do the trick here. Even if I 
would still like to see the automatic repartitioning optional since I 
would just reshuffle again. With windowed store I am a little bit 
sceptical about how to determine the window. So esentially one could run 
into problems when the rapid change happens near a window border. I will 
check you implementation in detail, if its problematic, we could still 
check _all_ windows on read with not to bad performance impact I guess. 
Will let you know if the implementation would be correct as is. I 
wouldn't not like to assume that: offset(A) < offset(B) => timestamp(A)  
< timestamp(B). I think we can't expect that.
>
>
> @Jan
> I believe I understand what you mean now - thanks for the diagram, it 
> did really help. You are correct that I do not have the original 
> primary key available, and I can see that if it was available then you 
> would be able to add and remove events from the Map. That being said, 
> I encourage you to finish your diagrams / charts just for clarity for 
> everyone else.
>
Yeah 100%, this giphy thing is just really hard work. But I understand 
the benefits for the rest. Sorry about the original primary key, We have 
join and Group by implemented our own in PAPI and basically not using 
any DSL (Just the abstraction). Completely missed that in original DSL 
its not there and just assumed it. total brain mess up on my end. Will 
finish the chart as soon as i get a quite evening this week.

> My follow up question for you is, won't the Map stay inside the State 
> Store indefinitely after all of the changes have propagated? Isn't 
> this effectively the same as a highwater mark state store?
Thing is that if the map is empty, substractor is gonna return `null` 
and the key is removed from the keyspace. But there is going to be a 
store 100%, the good thing is that I can use this store directly for
materialize() / enableSendingOldValues() is a regular store, satisfying 
all gurantees needed for further groupby / join. The Windowed store is 
not keeping the values, so for the next statefull operation we would
need to instantiate an extra store. or we have the window store also 
have the values then.

Long story short. if we can flip in a custom group by before 
repartitioning to the original primary key i think it would help the 
users big time in building efficient apps. Given the original primary 
key issue I understand that we do not have a solid foundation to build on.
Leaving primary key carry along to the user. very unfortunate. I could 
understand the decision goes like that. I do not think its a good decision.

>
>
>
>
> Thanks
> Adam
>
>
>
>
>
>
> On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre 
> <dumbreprajakta311@gmail.com <ma...@gmail.com>> wrote:
>
>     please remove me from this group
>
>     On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak
>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>     wrote:
>
>     > Hi Adam,
>     >
>     > give me some time, will make such a chart. last time i didn't
>     get along
>     > well with giphy and ruined all your charts.
>     > Hopefully i can get it done today
>     >
>     > On 08.09.2018 16:00, Adam Bellemare wrote:
>     > > Hi Jan
>     > >
>     > > I have included a diagram of what I attempted on the KIP.
>     > >
>     >
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-GroupBy+Reduce/Aggregate
>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-GroupBy+Reduce/Aggregate>
>     > >
>     > > I attempted this back at the start of my own implementation of
>     this
>     > > solution, and since I could not get it to work I have since
>     discarded the
>     > > code. At this point in time, if you wish to continue pursuing
>     for your
>     > > groupBy solution, I ask that you please create a diagram on
>     the KIP
>     > > carefully explaining your solution. Please feel free to use
>     the image I
>     > > just posted as a starting point. I am having trouble
>     understanding your
>     > > explanations but I think that a carefully constructed diagram
>     will clear
>     > up
>     > > any misunderstandings. Alternately, please post a
>     comprehensive PR with
>     > > your solution. I can only guess at what you mean, and since I
>     value my
>     > own
>     > > time as much as you value yours, I believe it is your
>     responsibility to
>     > > provide an implementation instead of me trying to guess.
>     > >
>     > > Adam
>     > >
>     > >
>     > >
>     > >
>     > >
>     > >
>     > >
>     > >
>     > >
>     > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak
>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>     > > wrote:
>     > >
>     > >> Hi James,
>     > >>
>     > >> nice to see you beeing interested. kafka streams at this
>     point supports
>     > >> all sorts of joins as long as both streams have the same key.
>     > >> Adam is currently implementing a join where a KTable and a
>     KTable can
>     > have
>     > >> a one to many relation ship (1:n). We exploit that rocksdb is a
>     > >> datastore that keeps data sorted (At least exposes an API to
>     access the
>     > >> stored data in a sorted fashion).
>     > >>
>     > >> I think the technical caveats are well understood now and we are
>     > basically
>     > >> down to philosophy and API Design ( when Adam sees my newest
>     message).
>     > >> I have a lengthy track record of loosing those kinda
>     arguments within
>     > the
>     > >> streams community and I have no clue why. So I literally
>     can't wait for
>     > you
>     > >> to churn through this thread and give you opinion on how we
>     should
>     > design
>     > >> the return type of the oneToManyJoin and how many power we
>     want to give
>     > to
>     > >> the user vs "simplicity" (where simplicity isn't really that
>     as users
>     > still
>     > >> need to understand it I argue)
>     > >>
>     > >> waiting for you to join in on the discussion
>     > >>
>     > >> Best Jan
>     > >>
>     > >>
>     > >>
>     > >> On 07.09.2018 15:49, James Kwan wrote:
>     > >>
>     > >>> I am new to this group and I found this subject
>     interesting.  Sounds
>     > like
>     > >>> you guys want to implement a join table of two streams? Is there
>     > somewhere
>     > >>> I can see the original requirement or proposal?
>     > >>>
>     > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak
>     <Jan.Filipiak@trivago.com <ma...@trivago.com>>
>     > >>>> wrote:
>     > >>>>
>     > >>>>
>     > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>     > >>>>
>     > >>>>> I'm currently testing using a Windowed Store to store the
>     highwater
>     > >>>>> mark.
>     > >>>>> By all indications this should work fine, with the caveat
>     being that
>     > it
>     > >>>>> can
>     > >>>>> only resolve out-of-order arrival for up to the size of
>     the window
>     > (ie:
>     > >>>>> 24h, 72h, etc). This would remove the possibility of it being
>     > unbounded
>     > >>>>> in
>     > >>>>> size.
>     > >>>>>
>     > >>>>> With regards to Jan's suggestion, I believe this is where
>     we will
>     > have
>     > >>>>> to
>     > >>>>> remain in disagreement. While I do not disagree with your
>     statement
>     > >>>>> about
>     > >>>>> there likely to be additional joins done in a real-world
>     workflow, I
>     > do
>     > >>>>> not
>     > >>>>> see how you can conclusively deal with out-of-order arrival of
>     > >>>>> foreign-key
>     > >>>>> changes and subsequent joins. I have attempted what I
>     think you have
>     > >>>>> proposed (without a high-water, using groupBy and reduce)
>     and found
>     > >>>>> that if
>     > >>>>> the foreign key changes too quickly, or the load on a
>     stream thread
>     > is
>     > >>>>> too
>     > >>>>> high, the joined messages will arrive out-of-order and be
>     incorrectly
>     > >>>>> propagated, such that an intermediate event is represented
>     as the
>     > final
>     > >>>>> event.
>     > >>>>>
>     > >>>> Can you shed some light on your groupBy implementation.
>     There must be
>     > >>>> some sort of flaw in it.
>     > >>>> I have a suspicion where it is, I would just like to
>     confirm. The idea
>     > >>>> is bullet proof and it must be
>     > >>>> an implementation mess up. I would like to clarify before
>     we draw a
>     > >>>> conclusion.
>     > >>>>
>     > >>>>    Repartitioning the scattered events back to their original
>     > >>>>> partitions is the only way I know how to conclusively deal
>     with
>     > >>>>> out-of-order events in a given time frame, and to ensure
>     that the
>     > data
>     > >>>>> is
>     > >>>>> eventually consistent with the input events.
>     > >>>>>
>     > >>>>> If you have some code to share that illustrates your
>     approach, I
>     > would
>     > >>>>> be
>     > >>>>> very grateful as it would remove any misunderstandings
>     that I may
>     > have.
>     > >>>>>
>     > >>>> ah okay you were looking for my code. I don't have
>     something easily
>     > >>>> readable here as its bloated with OO-patterns.
>     > >>>>
>     > >>>> its anyhow trivial:
>     > >>>>
>     > >>>> @Override
>     > >>>>      public T apply(K aggKey, V value, T aggregate)
>     > >>>>      {
>     > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
>     imaginary
>     > >>>>          U toModifyKey = mapper.apply(value);
>     > >>>>              << this is the place where people actually
>     gonna have
>     > issues
>     > >>>> and why you probably couldn't do it. we would need to find
>     a solution
>     > here.
>     > >>>> I didn't realize that yet.
>     > >>>>              << we propagate the field in the joiner, so
>     that we can
>     > pick
>     > >>>> it up in an aggregate. Probably you have not thought of
>     this in your
>     > >>>> approach right?
>     > >>>>              << I am very open to find a generic solution
>     here. In my
>     > >>>> honest opinion this is broken in KTableImpl.GroupBy that it
>     looses
>     > the keys
>     > >>>> and only maintains the aggregate key.
>     > >>>>              << I abstracted it away back then way before i was
>     > thinking
>     > >>>> of oneToMany join. That is why I didn't realize its
>     significance here.
>     > >>>>              << Opinions?
>     > >>>>
>     > >>>>          for (V m : current)
>     > >>>>          {
>     > >>>> currentStateAsMap.put(mapper.apply(m), m);
>     > >>>>          }
>     > >>>>          if (isAdder)
>     > >>>>          {
>     > >>>> currentStateAsMap.put(toModifyKey, value);
>     > >>>>          }
>     > >>>>          else
>     > >>>>          {
>     > >>>> currentStateAsMap.remove(toModifyKey);
>     > >>>> if(currentStateAsMap.isEmpty()){
>     > >>>>                  return null;
>     > >>>>              }
>     > >>>>          }
>     > >>>>          retrun asAggregateType(currentStateAsMap)
>     > >>>>      }
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>> Thanks,
>     > >>>>> Adam
>     > >>>>>
>     > >>>>>
>     > >>>>>
>     > >>>>>
>     > >>>>>
>     > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
>     > Jan.Filipiak@trivago.com <ma...@trivago.com>>
>     > >>>>> wrote:
>     > >>>>>
>     > >>>>> Thanks Adam for bringing Matthias to speed!
>     > >>>>>> about the differences. I think re-keying back should be
>     optional at
>     > >>>>>> best.
>     > >>>>>> I would say we return a KScatteredTable with reshuffle()
>     returning
>     > >>>>>> KTable<originalKey,Joined> to make the backwards
>     repartitioning
>     > >>>>>> optional.
>     > >>>>>> I am also in a big favour of doing the out of order
>     processing using
>     > >>>>>> group
>     > >>>>>> by instead high water mark tracking.
>     > >>>>>> Just because unbounded growth is just scary + It saves us
>     the header
>     > >>>>>> stuff.
>     > >>>>>>
>     > >>>>>> I think the abstraction of always repartitioning back is
>     just not so
>     > >>>>>> strong. Like the work has been done before we partition
>     back and
>     > >>>>>> grouping
>     > >>>>>> by something else afterwards is really common.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>     > >>>>>>
>     > >>>>>> Hi Matthias
>     > >>>>>>> Thank you for your feedback, I do appreciate it!
>     > >>>>>>>
>     > >>>>>>> While name spacing would be possible, it would require to
>     > deserialize
>     > >>>>>>>
>     > >>>>>>>> user headers what implies a runtime overhead. I would
>     suggest to
>     > no
>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>     > problem in
>     > >>>>>>>> the future, we can still add name spacing later on.
>     > >>>>>>>>
>     > >>>>>>>> Agreed. I will go with using a reserved string and
>     document it.
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> My main concern about the design it the type of the
>     result KTable:
>     > If
>     > >>>>>>> I
>     > >>>>>>> understood the proposal correctly,
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> In your example, you have table1 and table2 swapped.
>     Here is how it
>     > >>>>>>> works
>     > >>>>>>> currently:
>     > >>>>>>>
>     > >>>>>>> 1) table1 has the records that contain the foreign key
>     within their
>     > >>>>>>> value.
>     > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>     > >>>>>>> <c,(fk=B,bar=3)>
>     > >>>>>>> table2 input stream: <A,X>, <B,Y>
>     > >>>>>>>
>     > >>>>>>> 2) A Value mapper is required to extract the foreign key.
>     > >>>>>>> table1 foreign key mapper: ( value => value.fk
>     <http://value.fk> )
>     > >>>>>>>
>     > >>>>>>> The mapper is applied to each element in table1, and a
>     new combined
>     > >>>>>>> key is
>     > >>>>>>> made:
>     > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>,
>     <B-c,
>     > >>>>>>> (fk=B,bar=3)>
>     > >>>>>>>
>     > >>>>>>> 3) The rekeyed events are copartitioned with table2:
>     > >>>>>>>
>     > >>>>>>> a) Stream Thread with Partition 0:
>     > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b,
>     (fk=A,bar=2)>
>     > >>>>>>> Table2: <A,X>
>     > >>>>>>>
>     > >>>>>>> b) Stream Thread with Partition 1:
>     > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>     > >>>>>>> Table2: <B,Y>
>     > >>>>>>>
>     > >>>>>>> 4) From here, they can be joined together locally by
>     applying the
>     > >>>>>>> joiner
>     > >>>>>>> function.
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> At this point, Jan's design and my design deviate. My
>     design goes
>     > on
>     > >>>>>>> to
>     > >>>>>>> repartition the data post-join and resolve out-of-order
>     arrival of
>     > >>>>>>> records,
>     > >>>>>>> finally returning the data keyed just the original key.
>     I do not
>     > >>>>>>> expose
>     > >>>>>>> the
>     > >>>>>>> CombinedKey or any of the internals outside of the
>     joinOnForeignKey
>     > >>>>>>> function. This does make for larger footprint, but it
>     removes all
>     > >>>>>>> agency
>     > >>>>>>> for resolving out-of-order arrivals and handling
>     CombinedKeys from
>     > the
>     > >>>>>>> user. I believe that this makes the function much easier
>     to use.
>     > >>>>>>>
>     > >>>>>>> Let me know if this helps resolve your questions, and
>     please feel
>     > >>>>>>> free to
>     > >>>>>>> add anything else on your mind.
>     > >>>>>>>
>     > >>>>>>> Thanks again,
>     > >>>>>>> Adam
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>     > >>>>>>> matthias@confluent.io <ma...@confluent.io>>
>     > >>>>>>> wrote:
>     > >>>>>>>
>     > >>>>>>> Hi,
>     > >>>>>>>
>     > >>>>>>>> I am just catching up on this thread. I did not read
>     everything so
>     > >>>>>>>> far,
>     > >>>>>>>> but want to share couple of initial thoughts:
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> Headers: I think there is a fundamental difference
>     between header
>     > >>>>>>>> usage
>     > >>>>>>>> in this KIP and KP-258. For 258, we add headers to
>     changelog topic
>     > >>>>>>>> that
>     > >>>>>>>> are owned by Kafka Streams and nobody else is supposed
>     to write
>     > into
>     > >>>>>>>> them. In fact, no user header are written into the
>     changelog topic
>     > >>>>>>>> and
>     > >>>>>>>> thus, there are not conflicts.
>     > >>>>>>>>
>     > >>>>>>>> Nevertheless, I don't see a big issue with using
>     headers within
>     > >>>>>>>> Streams.
>     > >>>>>>>> As long as we document it, we can have some "reserved"
>     header keys
>     > >>>>>>>> and
>     > >>>>>>>> users are not allowed to use when processing data with
>     Kafka
>     > Streams.
>     > >>>>>>>> IMHO, this should be ok.
>     > >>>>>>>>
>     > >>>>>>>> I think there is a safe way to avoid conflicts, since these
>     > headers
>     > >>>>>>>> are
>     > >>>>>>>>
>     > >>>>>>>>> only needed in internal topics (I think):
>     > >>>>>>>>> For internal and changelog topics, we can namespace
>     all headers:
>     > >>>>>>>>> * user-defined headers are namespaced as "external." +
>     headerKey
>     > >>>>>>>>> * internal headers are namespaced as "internal." +
>     headerKey
>     > >>>>>>>>>
>     > >>>>>>>>> While name spacing would be possible, it would require to
>     > >>>>>>>> deserialize
>     > >>>>>>>> user headers what implies a runtime overhead. I would
>     suggest to
>     > no
>     > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
>     > problem in
>     > >>>>>>>> the future, we can still add name spacing later on.
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> My main concern about the design it the type of the
>     result KTable:
>     > >>>>>>>> If I
>     > >>>>>>>> understood the proposal correctly,
>     > >>>>>>>>
>     > >>>>>>>> KTable<K1,V1> table1 = ...
>     > >>>>>>>> KTable<K2,V2> table2 = ...
>     > >>>>>>>>
>     > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>     > >>>>>>>>
>     > >>>>>>>> implies that the `joinedTable` has the same key as the
>     left input
>     > >>>>>>>> table.
>     > >>>>>>>> IMHO, this does not work because if table2 contains
>     multiple rows
>     > >>>>>>>> that
>     > >>>>>>>> join with a record in table1 (what is the main purpose of a
>     > foreign
>     > >>>>>>>> key
>     > >>>>>>>> join), the result table would only contain a single
>     join result,
>     > but
>     > >>>>>>>> not
>     > >>>>>>>> multiple.
>     > >>>>>>>>
>     > >>>>>>>> Example:
>     > >>>>>>>>
>     > >>>>>>>> table1 input stream: <A,X>
>     > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>     > >>>>>>>>
>     > >>>>>>>> We use table2 value a foreign key to table1 key (ie,
>     "A" joins).
>     > If
>     > >>>>>>>> the
>     > >>>>>>>> result key is the same key as key of table1, this
>     implies that the
>     > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)>
>     but not
>     > both.
>     > >>>>>>>> Because the share the same key, whatever result record
>     we emit
>     > later,
>     > >>>>>>>> overwrite the previous result.
>     > >>>>>>>>
>     > >>>>>>>> This is the reason why Jan originally proposed to use a
>     > combination
>     > >>>>>>>> of
>     > >>>>>>>> both primary keys of the input tables as key of the
>     output table.
>     > >>>>>>>> This
>     > >>>>>>>> makes the keys of the output table unique and we can
>     store both in
>     > >>>>>>>> the
>     > >>>>>>>> output table:
>     > >>>>>>>>
>     > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> Thoughts?
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> -Matthias
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>     > >>>>>>>>
>     > >>>>>>>> Just on remark here.
>     > >>>>>>>>> The high-watermark could be disregarded. The decision
>     about the
>     > >>>>>>>>> forward
>     > >>>>>>>>> depends on the size of the aggregated map.
>     > >>>>>>>>> Only 1 element long maps would be unpacked and
>     forwarded. 0
>     > element
>     > >>>>>>>>> maps
>     > >>>>>>>>> would be published as delete. Any other count
>     > >>>>>>>>> of map entries is in "waiting for correct deletes to
>     > arrive"-state.
>     > >>>>>>>>>
>     > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>     > >>>>>>>>>
>     > >>>>>>>>> It does look like I could replace the second
>     repartition store
>     > and
>     > >>>>>>>>>> highwater store with a groupBy and reduce.  However,
>     it looks
>     > like
>     > >>>>>>>>>> I
>     > >>>>>>>>>> would
>     > >>>>>>>>>> still need to store the highwater value within the
>     materialized
>     > >>>>>>>>>> store,
>     > >>>>>>>>>>
>     > >>>>>>>>>> to
>     > >>>>>>>>> compare the arrival of out-of-order records (assuming my
>     > >>>>>>>>> understanding
>     > >>>>>>>>> of
>     > >>>>>>>>> THIS is correct...). This in effect is the same as the
>     design I
>     > have
>     > >>>>>>>>> now,
>     > >>>>>>>>> just with the two tables merged together.
>     > >>>>>>>>>
>     >
>     >
>
>


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Guozhang

Current highwater mark implementation would grow endlessly based on primary
key of original event. It is a pair of (<this table primary key>, <highest
offset seen for that key>). This is used to differentiate between late
arrivals and new updates. My newest proposal would be to replace it with a
Windowed state store of Duration N. This would allow the same behaviour,
but cap the size based on time. This should allow for all late-arriving
events to be processed, and should be customizable by the user to tailor to
their own needs (ie: perhaps just 10 minutes of window, or perhaps 7
days...).


@Jan
I believe I understand what you mean now - thanks for the diagram, it did
really help. You are correct that I do not have the original primary key
available, and I can see that if it was available then you would be able to
add and remove events from the Map. That being said, I encourage you to
finish your diagrams / charts just for clarity for everyone else.

My follow up question for you is, won't the Map stay inside the State Store
indefinitely after all of the changes have propagated? Isn't this
effectively the same as a highwater mark state store?




Thanks
Adam






On Tue, Sep 11, 2018 at 10:07 AM, Prajakta Dumbre <
dumbreprajakta311@gmail.com> wrote:

> please remove me from this group
>
> On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > Hi Adam,
> >
> > give me some time, will make such a chart. last time i didn't get along
> > well with giphy and ruined all your charts.
> > Hopefully i can get it done today
> >
> > On 08.09.2018 16:00, Adam Bellemare wrote:
> > > Hi Jan
> > >
> > > I have included a diagram of what I attempted on the KIP.
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 213+Support+non-key+joining+in+KTable#KIP-213Supportnon-
> keyjoininginKTable-GroupBy+Reduce/Aggregate
> > >
> > > I attempted this back at the start of my own implementation of this
> > > solution, and since I could not get it to work I have since discarded
> the
> > > code. At this point in time, if you wish to continue pursuing for your
> > > groupBy solution, I ask that you please create a diagram on the KIP
> > > carefully explaining your solution. Please feel free to use the image I
> > > just posted as a starting point. I am having trouble understanding your
> > > explanations but I think that a carefully constructed diagram will
> clear
> > up
> > > any misunderstandings. Alternately, please post a comprehensive PR with
> > > your solution. I can only guess at what you mean, and since I value my
> > own
> > > time as much as you value yours, I believe it is your responsibility to
> > > provide an implementation instead of me trying to guess.
> > >
> > > Adam
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak <Jan.Filipiak@trivago.com
> >
> > > wrote:
> > >
> > >> Hi James,
> > >>
> > >> nice to see you beeing interested. kafka streams at this point
> supports
> > >> all sorts of joins as long as both streams have the same key.
> > >> Adam is currently implementing a join where a KTable and a KTable can
> > have
> > >> a one to many relation ship (1:n). We exploit that rocksdb is a
> > >> datastore that keeps data sorted (At least exposes an API to access
> the
> > >> stored data in a sorted fashion).
> > >>
> > >> I think the technical caveats are well understood now and we are
> > basically
> > >> down to philosophy and API Design ( when Adam sees my newest message).
> > >> I have a lengthy track record of loosing those kinda arguments within
> > the
> > >> streams community and I have no clue why. So I literally can't wait
> for
> > you
> > >> to churn through this thread and give you opinion on how we should
> > design
> > >> the return type of the oneToManyJoin and how many power we want to
> give
> > to
> > >> the user vs "simplicity" (where simplicity isn't really that as users
> > still
> > >> need to understand it I argue)
> > >>
> > >> waiting for you to join in on the discussion
> > >>
> > >> Best Jan
> > >>
> > >>
> > >>
> > >> On 07.09.2018 15:49, James Kwan wrote:
> > >>
> > >>> I am new to this group and I found this subject interesting.  Sounds
> > like
> > >>> you guys want to implement a join table of two streams? Is there
> > somewhere
> > >>> I can see the original requirement or proposal?
> > >>>
> > >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
> > >>>> wrote:
> > >>>>
> > >>>>
> > >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> > >>>>
> > >>>>> I'm currently testing using a Windowed Store to store the highwater
> > >>>>> mark.
> > >>>>> By all indications this should work fine, with the caveat being
> that
> > it
> > >>>>> can
> > >>>>> only resolve out-of-order arrival for up to the size of the window
> > (ie:
> > >>>>> 24h, 72h, etc). This would remove the possibility of it being
> > unbounded
> > >>>>> in
> > >>>>> size.
> > >>>>>
> > >>>>> With regards to Jan's suggestion, I believe this is where we will
> > have
> > >>>>> to
> > >>>>> remain in disagreement. While I do not disagree with your statement
> > >>>>> about
> > >>>>> there likely to be additional joins done in a real-world workflow,
> I
> > do
> > >>>>> not
> > >>>>> see how you can conclusively deal with out-of-order arrival of
> > >>>>> foreign-key
> > >>>>> changes and subsequent joins. I have attempted what I think you
> have
> > >>>>> proposed (without a high-water, using groupBy and reduce) and found
> > >>>>> that if
> > >>>>> the foreign key changes too quickly, or the load on a stream thread
> > is
> > >>>>> too
> > >>>>> high, the joined messages will arrive out-of-order and be
> incorrectly
> > >>>>> propagated, such that an intermediate event is represented as the
> > final
> > >>>>> event.
> > >>>>>
> > >>>> Can you shed some light on your groupBy implementation. There must
> be
> > >>>> some sort of flaw in it.
> > >>>> I have a suspicion where it is, I would just like to confirm. The
> idea
> > >>>> is bullet proof and it must be
> > >>>> an implementation mess up. I would like to clarify before we draw a
> > >>>> conclusion.
> > >>>>
> > >>>>    Repartitioning the scattered events back to their original
> > >>>>> partitions is the only way I know how to conclusively deal with
> > >>>>> out-of-order events in a given time frame, and to ensure that the
> > data
> > >>>>> is
> > >>>>> eventually consistent with the input events.
> > >>>>>
> > >>>>> If you have some code to share that illustrates your approach, I
> > would
> > >>>>> be
> > >>>>> very grateful as it would remove any misunderstandings that I may
> > have.
> > >>>>>
> > >>>> ah okay you were looking for my code. I don't have something easily
> > >>>> readable here as its bloated with OO-patterns.
> > >>>>
> > >>>> its anyhow trivial:
> > >>>>
> > >>>> @Override
> > >>>>      public T apply(K aggKey, V value, T aggregate)
> > >>>>      {
> > >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); <<
> imaginary
> > >>>>          U toModifyKey = mapper.apply(value);
> > >>>>              << this is the place where people actually gonna have
> > issues
> > >>>> and why you probably couldn't do it. we would need to find a
> solution
> > here.
> > >>>> I didn't realize that yet.
> > >>>>              << we propagate the field in the joiner, so that we can
> > pick
> > >>>> it up in an aggregate. Probably you have not thought of this in your
> > >>>> approach right?
> > >>>>              << I am very open to find a generic solution here. In
> my
> > >>>> honest opinion this is broken in KTableImpl.GroupBy that it looses
> > the keys
> > >>>> and only maintains the aggregate key.
> > >>>>              << I abstracted it away back then way before i was
> > thinking
> > >>>> of oneToMany join. That is why I didn't realize its significance
> here.
> > >>>>              << Opinions?
> > >>>>
> > >>>>          for (V m : current)
> > >>>>          {
> > >>>>              currentStateAsMap.put(mapper.apply(m), m);
> > >>>>          }
> > >>>>          if (isAdder)
> > >>>>          {
> > >>>>              currentStateAsMap.put(toModifyKey, value);
> > >>>>          }
> > >>>>          else
> > >>>>          {
> > >>>>              currentStateAsMap.remove(toModifyKey);
> > >>>>              if(currentStateAsMap.isEmpty()){
> > >>>>                  return null;
> > >>>>              }
> > >>>>          }
> > >>>>          retrun asAggregateType(currentStateAsMap)
> > >>>>      }
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>> Thanks,
> > >>>>> Adam
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> > Jan.Filipiak@trivago.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Thanks Adam for bringing Matthias to speed!
> > >>>>>> about the differences. I think re-keying back should be optional
> at
> > >>>>>> best.
> > >>>>>> I would say we return a KScatteredTable with reshuffle() returning
> > >>>>>> KTable<originalKey,Joined> to make the backwards repartitioning
> > >>>>>> optional.
> > >>>>>> I am also in a big favour of doing the out of order processing
> using
> > >>>>>> group
> > >>>>>> by instead high water mark tracking.
> > >>>>>> Just because unbounded growth is just scary + It saves us the
> header
> > >>>>>> stuff.
> > >>>>>>
> > >>>>>> I think the abstraction of always repartitioning back is just not
> so
> > >>>>>> strong. Like the work has been done before we partition back and
> > >>>>>> grouping
> > >>>>>> by something else afterwards is really common.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> > >>>>>>
> > >>>>>> Hi Matthias
> > >>>>>>> Thank you for your feedback, I do appreciate it!
> > >>>>>>>
> > >>>>>>> While name spacing would be possible, it would require to
> > deserialize
> > >>>>>>>
> > >>>>>>>> user headers what implies a runtime overhead. I would suggest to
> > no
> > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
> > problem in
> > >>>>>>>> the future, we can still add name spacing later on.
> > >>>>>>>>
> > >>>>>>>> Agreed. I will go with using a reserved string and document it.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> My main concern about the design it the type of the result
> KTable:
> > If
> > >>>>>>> I
> > >>>>>>> understood the proposal correctly,
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> In your example, you have table1 and table2 swapped. Here is how
> it
> > >>>>>>> works
> > >>>>>>> currently:
> > >>>>>>>
> > >>>>>>> 1) table1 has the records that contain the foreign key within
> their
> > >>>>>>> value.
> > >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
> > >>>>>>> <c,(fk=B,bar=3)>
> > >>>>>>> table2 input stream: <A,X>, <B,Y>
> > >>>>>>>
> > >>>>>>> 2) A Value mapper is required to extract the foreign key.
> > >>>>>>> table1 foreign key mapper: ( value => value.fk )
> > >>>>>>>
> > >>>>>>> The mapper is applied to each element in table1, and a new
> combined
> > >>>>>>> key is
> > >>>>>>> made:
> > >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
> > >>>>>>> (fk=B,bar=3)>
> > >>>>>>>
> > >>>>>>> 3) The rekeyed events are copartitioned with table2:
> > >>>>>>>
> > >>>>>>> a) Stream Thread with Partition 0:
> > >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
> > >>>>>>> Table2: <A,X>
> > >>>>>>>
> > >>>>>>> b) Stream Thread with Partition 1:
> > >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> > >>>>>>> Table2: <B,Y>
> > >>>>>>>
> > >>>>>>> 4) From here, they can be joined together locally by applying the
> > >>>>>>> joiner
> > >>>>>>> function.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> At this point, Jan's design and my design deviate. My design goes
> > on
> > >>>>>>> to
> > >>>>>>> repartition the data post-join and resolve out-of-order arrival
> of
> > >>>>>>> records,
> > >>>>>>> finally returning the data keyed just the original key. I do not
> > >>>>>>> expose
> > >>>>>>> the
> > >>>>>>> CombinedKey or any of the internals outside of the
> joinOnForeignKey
> > >>>>>>> function. This does make for larger footprint, but it removes all
> > >>>>>>> agency
> > >>>>>>> for resolving out-of-order arrivals and handling CombinedKeys
> from
> > the
> > >>>>>>> user. I believe that this makes the function much easier to use.
> > >>>>>>>
> > >>>>>>> Let me know if this helps resolve your questions, and please feel
> > >>>>>>> free to
> > >>>>>>> add anything else on your mind.
> > >>>>>>>
> > >>>>>>> Thanks again,
> > >>>>>>> Adam
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> > >>>>>>> matthias@confluent.io>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>> Hi,
> > >>>>>>>
> > >>>>>>>> I am just catching up on this thread. I did not read everything
> so
> > >>>>>>>> far,
> > >>>>>>>> but want to share couple of initial thoughts:
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Headers: I think there is a fundamental difference between
> header
> > >>>>>>>> usage
> > >>>>>>>> in this KIP and KP-258. For 258, we add headers to changelog
> topic
> > >>>>>>>> that
> > >>>>>>>> are owned by Kafka Streams and nobody else is supposed to write
> > into
> > >>>>>>>> them. In fact, no user header are written into the changelog
> topic
> > >>>>>>>> and
> > >>>>>>>> thus, there are not conflicts.
> > >>>>>>>>
> > >>>>>>>> Nevertheless, I don't see a big issue with using headers within
> > >>>>>>>> Streams.
> > >>>>>>>> As long as we document it, we can have some "reserved" header
> keys
> > >>>>>>>> and
> > >>>>>>>> users are not allowed to use when processing data with Kafka
> > Streams.
> > >>>>>>>> IMHO, this should be ok.
> > >>>>>>>>
> > >>>>>>>> I think there is a safe way to avoid conflicts, since these
> > headers
> > >>>>>>>> are
> > >>>>>>>>
> > >>>>>>>>> only needed in internal topics (I think):
> > >>>>>>>>> For internal and changelog topics, we can namespace all
> headers:
> > >>>>>>>>> * user-defined headers are namespaced as "external." +
> headerKey
> > >>>>>>>>> * internal headers are namespaced as "internal." + headerKey
> > >>>>>>>>>
> > >>>>>>>>> While name spacing would be possible, it would require to
> > >>>>>>>> deserialize
> > >>>>>>>> user headers what implies a runtime overhead. I would suggest to
> > no
> > >>>>>>>> namespace for now to avoid the overhead. If this becomes a
> > problem in
> > >>>>>>>> the future, we can still add name spacing later on.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> My main concern about the design it the type of the result
> KTable:
> > >>>>>>>> If I
> > >>>>>>>> understood the proposal correctly,
> > >>>>>>>>
> > >>>>>>>> KTable<K1,V1> table1 = ...
> > >>>>>>>> KTable<K2,V2> table2 = ...
> > >>>>>>>>
> > >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
> > >>>>>>>>
> > >>>>>>>> implies that the `joinedTable` has the same key as the left
> input
> > >>>>>>>> table.
> > >>>>>>>> IMHO, this does not work because if table2 contains multiple
> rows
> > >>>>>>>> that
> > >>>>>>>> join with a record in table1 (what is the main purpose of a
> > foreign
> > >>>>>>>> key
> > >>>>>>>> join), the result table would only contain a single join result,
> > but
> > >>>>>>>> not
> > >>>>>>>> multiple.
> > >>>>>>>>
> > >>>>>>>> Example:
> > >>>>>>>>
> > >>>>>>>> table1 input stream: <A,X>
> > >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> > >>>>>>>>
> > >>>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins).
> > If
> > >>>>>>>> the
> > >>>>>>>> result key is the same key as key of table1, this implies that
> the
> > >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not
> > both.
> > >>>>>>>> Because the share the same key, whatever result record we emit
> > later,
> > >>>>>>>> overwrite the previous result.
> > >>>>>>>>
> > >>>>>>>> This is the reason why Jan originally proposed to use a
> > combination
> > >>>>>>>> of
> > >>>>>>>> both primary keys of the input tables as key of the output
> table.
> > >>>>>>>> This
> > >>>>>>>> makes the keys of the output table unique and we can store both
> in
> > >>>>>>>> the
> > >>>>>>>> output table:
> > >>>>>>>>
> > >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Thoughts?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> -Matthias
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> > >>>>>>>>
> > >>>>>>>> Just on remark here.
> > >>>>>>>>> The high-watermark could be disregarded. The decision about the
> > >>>>>>>>> forward
> > >>>>>>>>> depends on the size of the aggregated map.
> > >>>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0
> > element
> > >>>>>>>>> maps
> > >>>>>>>>> would be published as delete. Any other count
> > >>>>>>>>> of map entries is in "waiting for correct deletes to
> > arrive"-state.
> > >>>>>>>>>
> > >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> > >>>>>>>>>
> > >>>>>>>>> It does look like I could replace the second repartition store
> > and
> > >>>>>>>>>> highwater store with a groupBy and reduce.  However, it looks
> > like
> > >>>>>>>>>> I
> > >>>>>>>>>> would
> > >>>>>>>>>> still need to store the highwater value within the
> materialized
> > >>>>>>>>>> store,
> > >>>>>>>>>>
> > >>>>>>>>>> to
> > >>>>>>>>> compare the arrival of out-of-order records (assuming my
> > >>>>>>>>> understanding
> > >>>>>>>>> of
> > >>>>>>>>> THIS is correct...). This in effect is the same as the design I
> > have
> > >>>>>>>>> now,
> > >>>>>>>>> just with the two tables merged together.
> > >>>>>>>>>
> >
> >
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Prajakta Dumbre <du...@gmail.com>.
please remove me from this group

On Tue, Sep 11, 2018 at 1:29 PM Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi Adam,
>
> give me some time, will make such a chart. last time i didn't get along
> well with giphy and ruined all your charts.
> Hopefully i can get it done today
>
> On 08.09.2018 16:00, Adam Bellemare wrote:
> > Hi Jan
> >
> > I have included a diagram of what I attempted on the KIP.
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-GroupBy+Reduce/Aggregate
> >
> > I attempted this back at the start of my own implementation of this
> > solution, and since I could not get it to work I have since discarded the
> > code. At this point in time, if you wish to continue pursuing for your
> > groupBy solution, I ask that you please create a diagram on the KIP
> > carefully explaining your solution. Please feel free to use the image I
> > just posted as a starting point. I am having trouble understanding your
> > explanations but I think that a carefully constructed diagram will clear
> up
> > any misunderstandings. Alternately, please post a comprehensive PR with
> > your solution. I can only guess at what you mean, and since I value my
> own
> > time as much as you value yours, I believe it is your responsibility to
> > provide an implementation instead of me trying to guess.
> >
> > Adam
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> >> Hi James,
> >>
> >> nice to see you beeing interested. kafka streams at this point supports
> >> all sorts of joins as long as both streams have the same key.
> >> Adam is currently implementing a join where a KTable and a KTable can
> have
> >> a one to many relation ship (1:n). We exploit that rocksdb is a
> >> datastore that keeps data sorted (At least exposes an API to access the
> >> stored data in a sorted fashion).
> >>
> >> I think the technical caveats are well understood now and we are
> basically
> >> down to philosophy and API Design ( when Adam sees my newest message).
> >> I have a lengthy track record of loosing those kinda arguments within
> the
> >> streams community and I have no clue why. So I literally can't wait for
> you
> >> to churn through this thread and give you opinion on how we should
> design
> >> the return type of the oneToManyJoin and how many power we want to give
> to
> >> the user vs "simplicity" (where simplicity isn't really that as users
> still
> >> need to understand it I argue)
> >>
> >> waiting for you to join in on the discussion
> >>
> >> Best Jan
> >>
> >>
> >>
> >> On 07.09.2018 15:49, James Kwan wrote:
> >>
> >>> I am new to this group and I found this subject interesting.  Sounds
> like
> >>> you guys want to implement a join table of two streams? Is there
> somewhere
> >>> I can see the original requirement or proposal?
> >>>
> >>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
> >>>> wrote:
> >>>>
> >>>>
> >>>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>>
> >>>>> I'm currently testing using a Windowed Store to store the highwater
> >>>>> mark.
> >>>>> By all indications this should work fine, with the caveat being that
> it
> >>>>> can
> >>>>> only resolve out-of-order arrival for up to the size of the window
> (ie:
> >>>>> 24h, 72h, etc). This would remove the possibility of it being
> unbounded
> >>>>> in
> >>>>> size.
> >>>>>
> >>>>> With regards to Jan's suggestion, I believe this is where we will
> have
> >>>>> to
> >>>>> remain in disagreement. While I do not disagree with your statement
> >>>>> about
> >>>>> there likely to be additional joins done in a real-world workflow, I
> do
> >>>>> not
> >>>>> see how you can conclusively deal with out-of-order arrival of
> >>>>> foreign-key
> >>>>> changes and subsequent joins. I have attempted what I think you have
> >>>>> proposed (without a high-water, using groupBy and reduce) and found
> >>>>> that if
> >>>>> the foreign key changes too quickly, or the load on a stream thread
> is
> >>>>> too
> >>>>> high, the joined messages will arrive out-of-order and be incorrectly
> >>>>> propagated, such that an intermediate event is represented as the
> final
> >>>>> event.
> >>>>>
> >>>> Can you shed some light on your groupBy implementation. There must be
> >>>> some sort of flaw in it.
> >>>> I have a suspicion where it is, I would just like to confirm. The idea
> >>>> is bullet proof and it must be
> >>>> an implementation mess up. I would like to clarify before we draw a
> >>>> conclusion.
> >>>>
> >>>>    Repartitioning the scattered events back to their original
> >>>>> partitions is the only way I know how to conclusively deal with
> >>>>> out-of-order events in a given time frame, and to ensure that the
> data
> >>>>> is
> >>>>> eventually consistent with the input events.
> >>>>>
> >>>>> If you have some code to share that illustrates your approach, I
> would
> >>>>> be
> >>>>> very grateful as it would remove any misunderstandings that I may
> have.
> >>>>>
> >>>> ah okay you were looking for my code. I don't have something easily
> >>>> readable here as its bloated with OO-patterns.
> >>>>
> >>>> its anyhow trivial:
> >>>>
> >>>> @Override
> >>>>      public T apply(K aggKey, V value, T aggregate)
> >>>>      {
> >>>>          Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
> >>>>          U toModifyKey = mapper.apply(value);
> >>>>              << this is the place where people actually gonna have
> issues
> >>>> and why you probably couldn't do it. we would need to find a solution
> here.
> >>>> I didn't realize that yet.
> >>>>              << we propagate the field in the joiner, so that we can
> pick
> >>>> it up in an aggregate. Probably you have not thought of this in your
> >>>> approach right?
> >>>>              << I am very open to find a generic solution here. In my
> >>>> honest opinion this is broken in KTableImpl.GroupBy that it looses
> the keys
> >>>> and only maintains the aggregate key.
> >>>>              << I abstracted it away back then way before i was
> thinking
> >>>> of oneToMany join. That is why I didn't realize its significance here.
> >>>>              << Opinions?
> >>>>
> >>>>          for (V m : current)
> >>>>          {
> >>>>              currentStateAsMap.put(mapper.apply(m), m);
> >>>>          }
> >>>>          if (isAdder)
> >>>>          {
> >>>>              currentStateAsMap.put(toModifyKey, value);
> >>>>          }
> >>>>          else
> >>>>          {
> >>>>              currentStateAsMap.remove(toModifyKey);
> >>>>              if(currentStateAsMap.isEmpty()){
> >>>>                  return null;
> >>>>              }
> >>>>          }
> >>>>          retrun asAggregateType(currentStateAsMap)
> >>>>      }
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> Thanks,
> >>>>> Adam
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> Jan.Filipiak@trivago.com>
> >>>>> wrote:
> >>>>>
> >>>>> Thanks Adam for bringing Matthias to speed!
> >>>>>> about the differences. I think re-keying back should be optional at
> >>>>>> best.
> >>>>>> I would say we return a KScatteredTable with reshuffle() returning
> >>>>>> KTable<originalKey,Joined> to make the backwards repartitioning
> >>>>>> optional.
> >>>>>> I am also in a big favour of doing the out of order processing using
> >>>>>> group
> >>>>>> by instead high water mark tracking.
> >>>>>> Just because unbounded growth is just scary + It saves us the header
> >>>>>> stuff.
> >>>>>>
> >>>>>> I think the abstraction of always repartitioning back is just not so
> >>>>>> strong. Like the work has been done before we partition back and
> >>>>>> grouping
> >>>>>> by something else afterwards is really common.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>>>>
> >>>>>> Hi Matthias
> >>>>>>> Thank you for your feedback, I do appreciate it!
> >>>>>>>
> >>>>>>> While name spacing would be possible, it would require to
> deserialize
> >>>>>>>
> >>>>>>>> user headers what implies a runtime overhead. I would suggest to
> no
> >>>>>>>> namespace for now to avoid the overhead. If this becomes a
> problem in
> >>>>>>>> the future, we can still add name spacing later on.
> >>>>>>>>
> >>>>>>>> Agreed. I will go with using a reserved string and document it.
> >>>>>>>
> >>>>>>>
> >>>>>>> My main concern about the design it the type of the result KTable:
> If
> >>>>>>> I
> >>>>>>> understood the proposal correctly,
> >>>>>>>
> >>>>>>>
> >>>>>>> In your example, you have table1 and table2 swapped. Here is how it
> >>>>>>> works
> >>>>>>> currently:
> >>>>>>>
> >>>>>>> 1) table1 has the records that contain the foreign key within their
> >>>>>>> value.
> >>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
> >>>>>>> <c,(fk=B,bar=3)>
> >>>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>>
> >>>>>>> 2) A Value mapper is required to extract the foreign key.
> >>>>>>> table1 foreign key mapper: ( value => value.fk )
> >>>>>>>
> >>>>>>> The mapper is applied to each element in table1, and a new combined
> >>>>>>> key is
> >>>>>>> made:
> >>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
> >>>>>>> (fk=B,bar=3)>
> >>>>>>>
> >>>>>>> 3) The rekeyed events are copartitioned with table2:
> >>>>>>>
> >>>>>>> a) Stream Thread with Partition 0:
> >>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
> >>>>>>> Table2: <A,X>
> >>>>>>>
> >>>>>>> b) Stream Thread with Partition 1:
> >>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>>>>> Table2: <B,Y>
> >>>>>>>
> >>>>>>> 4) From here, they can be joined together locally by applying the
> >>>>>>> joiner
> >>>>>>> function.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> At this point, Jan's design and my design deviate. My design goes
> on
> >>>>>>> to
> >>>>>>> repartition the data post-join and resolve out-of-order arrival of
> >>>>>>> records,
> >>>>>>> finally returning the data keyed just the original key. I do not
> >>>>>>> expose
> >>>>>>> the
> >>>>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
> >>>>>>> function. This does make for larger footprint, but it removes all
> >>>>>>> agency
> >>>>>>> for resolving out-of-order arrivals and handling CombinedKeys from
> the
> >>>>>>> user. I believe that this makes the function much easier to use.
> >>>>>>>
> >>>>>>> Let me know if this helps resolve your questions, and please feel
> >>>>>>> free to
> >>>>>>> add anything else on your mind.
> >>>>>>>
> >>>>>>> Thanks again,
> >>>>>>> Adam
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> >>>>>>> matthias@confluent.io>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>> Hi,
> >>>>>>>
> >>>>>>>> I am just catching up on this thread. I did not read everything so
> >>>>>>>> far,
> >>>>>>>> but want to share couple of initial thoughts:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Headers: I think there is a fundamental difference between header
> >>>>>>>> usage
> >>>>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic
> >>>>>>>> that
> >>>>>>>> are owned by Kafka Streams and nobody else is supposed to write
> into
> >>>>>>>> them. In fact, no user header are written into the changelog topic
> >>>>>>>> and
> >>>>>>>> thus, there are not conflicts.
> >>>>>>>>
> >>>>>>>> Nevertheless, I don't see a big issue with using headers within
> >>>>>>>> Streams.
> >>>>>>>> As long as we document it, we can have some "reserved" header keys
> >>>>>>>> and
> >>>>>>>> users are not allowed to use when processing data with Kafka
> Streams.
> >>>>>>>> IMHO, this should be ok.
> >>>>>>>>
> >>>>>>>> I think there is a safe way to avoid conflicts, since these
> headers
> >>>>>>>> are
> >>>>>>>>
> >>>>>>>>> only needed in internal topics (I think):
> >>>>>>>>> For internal and changelog topics, we can namespace all headers:
> >>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
> >>>>>>>>> * internal headers are namespaced as "internal." + headerKey
> >>>>>>>>>
> >>>>>>>>> While name spacing would be possible, it would require to
> >>>>>>>> deserialize
> >>>>>>>> user headers what implies a runtime overhead. I would suggest to
> no
> >>>>>>>> namespace for now to avoid the overhead. If this becomes a
> problem in
> >>>>>>>> the future, we can still add name spacing later on.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> My main concern about the design it the type of the result KTable:
> >>>>>>>> If I
> >>>>>>>> understood the proposal correctly,
> >>>>>>>>
> >>>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>>
> >>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
> >>>>>>>>
> >>>>>>>> implies that the `joinedTable` has the same key as the left input
> >>>>>>>> table.
> >>>>>>>> IMHO, this does not work because if table2 contains multiple rows
> >>>>>>>> that
> >>>>>>>> join with a record in table1 (what is the main purpose of a
> foreign
> >>>>>>>> key
> >>>>>>>> join), the result table would only contain a single join result,
> but
> >>>>>>>> not
> >>>>>>>> multiple.
> >>>>>>>>
> >>>>>>>> Example:
> >>>>>>>>
> >>>>>>>> table1 input stream: <A,X>
> >>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>>>>
> >>>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins).
> If
> >>>>>>>> the
> >>>>>>>> result key is the same key as key of table1, this implies that the
> >>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not
> both.
> >>>>>>>> Because the share the same key, whatever result record we emit
> later,
> >>>>>>>> overwrite the previous result.
> >>>>>>>>
> >>>>>>>> This is the reason why Jan originally proposed to use a
> combination
> >>>>>>>> of
> >>>>>>>> both primary keys of the input tables as key of the output table.
> >>>>>>>> This
> >>>>>>>> makes the keys of the output table unique and we can store both in
> >>>>>>>> the
> >>>>>>>> output table:
> >>>>>>>>
> >>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Thoughts?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>>>>
> >>>>>>>> Just on remark here.
> >>>>>>>>> The high-watermark could be disregarded. The decision about the
> >>>>>>>>> forward
> >>>>>>>>> depends on the size of the aggregated map.
> >>>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0
> element
> >>>>>>>>> maps
> >>>>>>>>> would be published as delete. Any other count
> >>>>>>>>> of map entries is in "waiting for correct deletes to
> arrive"-state.
> >>>>>>>>>
> >>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>>>>
> >>>>>>>>> It does look like I could replace the second repartition store
> and
> >>>>>>>>>> highwater store with a groupBy and reduce.  However, it looks
> like
> >>>>>>>>>> I
> >>>>>>>>>> would
> >>>>>>>>>> still need to store the highwater value within the materialized
> >>>>>>>>>> store,
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>> compare the arrival of out-of-order records (assuming my
> >>>>>>>>> understanding
> >>>>>>>>> of
> >>>>>>>>> THIS is correct...). This in effect is the same as the design I
> have
> >>>>>>>>> now,
> >>>>>>>>> just with the two tables merged together.
> >>>>>>>>>
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

give me some time, will make such a chart. last time i didn't get along 
well with giphy and ruined all your charts.
Hopefully i can get it done today

On 08.09.2018 16:00, Adam Bellemare wrote:
> Hi Jan
>
> I have included a diagram of what I attempted on the KIP.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-GroupBy+Reduce/Aggregate
>
> I attempted this back at the start of my own implementation of this
> solution, and since I could not get it to work I have since discarded the
> code. At this point in time, if you wish to continue pursuing for your
> groupBy solution, I ask that you please create a diagram on the KIP
> carefully explaining your solution. Please feel free to use the image I
> just posted as a starting point. I am having trouble understanding your
> explanations but I think that a carefully constructed diagram will clear up
> any misunderstandings. Alternately, please post a comprehensive PR with
> your solution. I can only guess at what you mean, and since I value my own
> time as much as you value yours, I believe it is your responsibility to
> provide an implementation instead of me trying to guess.
>
> Adam
>
>
>
>
>
>
>
>
>
> On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Hi James,
>>
>> nice to see you beeing interested. kafka streams at this point supports
>> all sorts of joins as long as both streams have the same key.
>> Adam is currently implementing a join where a KTable and a KTable can have
>> a one to many relation ship (1:n). We exploit that rocksdb is a
>> datastore that keeps data sorted (At least exposes an API to access the
>> stored data in a sorted fashion).
>>
>> I think the technical caveats are well understood now and we are basically
>> down to philosophy and API Design ( when Adam sees my newest message).
>> I have a lengthy track record of loosing those kinda arguments within the
>> streams community and I have no clue why. So I literally can't wait for you
>> to churn through this thread and give you opinion on how we should design
>> the return type of the oneToManyJoin and how many power we want to give to
>> the user vs "simplicity" (where simplicity isn't really that as users still
>> need to understand it I argue)
>>
>> waiting for you to join in on the discussion
>>
>> Best Jan
>>
>>
>>
>> On 07.09.2018 15:49, James Kwan wrote:
>>
>>> I am new to this group and I found this subject interesting.  Sounds like
>>> you guys want to implement a join table of two streams? Is there somewhere
>>> I can see the original requirement or proposal?
>>>
>>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>
>>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>>
>>>>> I'm currently testing using a Windowed Store to store the highwater
>>>>> mark.
>>>>> By all indications this should work fine, with the caveat being that it
>>>>> can
>>>>> only resolve out-of-order arrival for up to the size of the window (ie:
>>>>> 24h, 72h, etc). This would remove the possibility of it being unbounded
>>>>> in
>>>>> size.
>>>>>
>>>>> With regards to Jan's suggestion, I believe this is where we will have
>>>>> to
>>>>> remain in disagreement. While I do not disagree with your statement
>>>>> about
>>>>> there likely to be additional joins done in a real-world workflow, I do
>>>>> not
>>>>> see how you can conclusively deal with out-of-order arrival of
>>>>> foreign-key
>>>>> changes and subsequent joins. I have attempted what I think you have
>>>>> proposed (without a high-water, using groupBy and reduce) and found
>>>>> that if
>>>>> the foreign key changes too quickly, or the load on a stream thread is
>>>>> too
>>>>> high, the joined messages will arrive out-of-order and be incorrectly
>>>>> propagated, such that an intermediate event is represented as the final
>>>>> event.
>>>>>
>>>> Can you shed some light on your groupBy implementation. There must be
>>>> some sort of flaw in it.
>>>> I have a suspicion where it is, I would just like to confirm. The idea
>>>> is bullet proof and it must be
>>>> an implementation mess up. I would like to clarify before we draw a
>>>> conclusion.
>>>>
>>>>    Repartitioning the scattered events back to their original
>>>>> partitions is the only way I know how to conclusively deal with
>>>>> out-of-order events in a given time frame, and to ensure that the data
>>>>> is
>>>>> eventually consistent with the input events.
>>>>>
>>>>> If you have some code to share that illustrates your approach, I would
>>>>> be
>>>>> very grateful as it would remove any misunderstandings that I may have.
>>>>>
>>>> ah okay you were looking for my code. I don't have something easily
>>>> readable here as its bloated with OO-patterns.
>>>>
>>>> its anyhow trivial:
>>>>
>>>> @Override
>>>>      public T apply(K aggKey, V value, T aggregate)
>>>>      {
>>>>          Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
>>>>          U toModifyKey = mapper.apply(value);
>>>>              << this is the place where people actually gonna have issues
>>>> and why you probably couldn't do it. we would need to find a solution here.
>>>> I didn't realize that yet.
>>>>              << we propagate the field in the joiner, so that we can pick
>>>> it up in an aggregate. Probably you have not thought of this in your
>>>> approach right?
>>>>              << I am very open to find a generic solution here. In my
>>>> honest opinion this is broken in KTableImpl.GroupBy that it looses the keys
>>>> and only maintains the aggregate key.
>>>>              << I abstracted it away back then way before i was thinking
>>>> of oneToMany join. That is why I didn't realize its significance here.
>>>>              << Opinions?
>>>>
>>>>          for (V m : current)
>>>>          {
>>>>              currentStateAsMap.put(mapper.apply(m), m);
>>>>          }
>>>>          if (isAdder)
>>>>          {
>>>>              currentStateAsMap.put(toModifyKey, value);
>>>>          }
>>>>          else
>>>>          {
>>>>              currentStateAsMap.remove(toModifyKey);
>>>>              if(currentStateAsMap.isEmpty()){
>>>>                  return null;
>>>>              }
>>>>          }
>>>>          retrun asAggregateType(currentStateAsMap)
>>>>      }
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> Thanks,
>>>>> Adam
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
>>>>> wrote:
>>>>>
>>>>> Thanks Adam for bringing Matthias to speed!
>>>>>> about the differences. I think re-keying back should be optional at
>>>>>> best.
>>>>>> I would say we return a KScatteredTable with reshuffle() returning
>>>>>> KTable<originalKey,Joined> to make the backwards repartitioning
>>>>>> optional.
>>>>>> I am also in a big favour of doing the out of order processing using
>>>>>> group
>>>>>> by instead high water mark tracking.
>>>>>> Just because unbounded growth is just scary + It saves us the header
>>>>>> stuff.
>>>>>>
>>>>>> I think the abstraction of always repartitioning back is just not so
>>>>>> strong. Like the work has been done before we partition back and
>>>>>> grouping
>>>>>> by something else afterwards is really common.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>>
>>>>>> Hi Matthias
>>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>>>
>>>>>>> While name spacing would be possible, it would require to deserialize
>>>>>>>
>>>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>>>> the future, we can still add name spacing later on.
>>>>>>>>
>>>>>>>> Agreed. I will go with using a reserved string and document it.
>>>>>>>
>>>>>>>
>>>>>>> My main concern about the design it the type of the result KTable: If
>>>>>>> I
>>>>>>> understood the proposal correctly,
>>>>>>>
>>>>>>>
>>>>>>> In your example, you have table1 and table2 swapped. Here is how it
>>>>>>> works
>>>>>>> currently:
>>>>>>>
>>>>>>> 1) table1 has the records that contain the foreign key within their
>>>>>>> value.
>>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>>>>>>> <c,(fk=B,bar=3)>
>>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>>
>>>>>>> 2) A Value mapper is required to extract the foreign key.
>>>>>>> table1 foreign key mapper: ( value => value.fk )
>>>>>>>
>>>>>>> The mapper is applied to each element in table1, and a new combined
>>>>>>> key is
>>>>>>> made:
>>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>>>>>>> (fk=B,bar=3)>
>>>>>>>
>>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>>>
>>>>>>> a) Stream Thread with Partition 0:
>>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>>>>>>> Table2: <A,X>
>>>>>>>
>>>>>>> b) Stream Thread with Partition 1:
>>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>>> Table2: <B,Y>
>>>>>>>
>>>>>>> 4) From here, they can be joined together locally by applying the
>>>>>>> joiner
>>>>>>> function.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> At this point, Jan's design and my design deviate. My design goes on
>>>>>>> to
>>>>>>> repartition the data post-join and resolve out-of-order arrival of
>>>>>>> records,
>>>>>>> finally returning the data keyed just the original key. I do not
>>>>>>> expose
>>>>>>> the
>>>>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
>>>>>>> function. This does make for larger footprint, but it removes all
>>>>>>> agency
>>>>>>> for resolving out-of-order arrivals and handling CombinedKeys from the
>>>>>>> user. I believe that this makes the function much easier to use.
>>>>>>>
>>>>>>> Let me know if this helps resolve your questions, and please feel
>>>>>>> free to
>>>>>>> add anything else on your mind.
>>>>>>>
>>>>>>> Thanks again,
>>>>>>> Adam
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>>> matthias@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>>> I am just catching up on this thread. I did not read everything so
>>>>>>>> far,
>>>>>>>> but want to share couple of initial thoughts:
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Headers: I think there is a fundamental difference between header
>>>>>>>> usage
>>>>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic
>>>>>>>> that
>>>>>>>> are owned by Kafka Streams and nobody else is supposed to write into
>>>>>>>> them. In fact, no user header are written into the changelog topic
>>>>>>>> and
>>>>>>>> thus, there are not conflicts.
>>>>>>>>
>>>>>>>> Nevertheless, I don't see a big issue with using headers within
>>>>>>>> Streams.
>>>>>>>> As long as we document it, we can have some "reserved" header keys
>>>>>>>> and
>>>>>>>> users are not allowed to use when processing data with Kafka Streams.
>>>>>>>> IMHO, this should be ok.
>>>>>>>>
>>>>>>>> I think there is a safe way to avoid conflicts, since these headers
>>>>>>>> are
>>>>>>>>
>>>>>>>>> only needed in internal topics (I think):
>>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>>
>>>>>>>>> While name spacing would be possible, it would require to
>>>>>>>> deserialize
>>>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>>>> the future, we can still add name spacing later on.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> My main concern about the design it the type of the result KTable:
>>>>>>>> If I
>>>>>>>> understood the proposal correctly,
>>>>>>>>
>>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>>
>>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>>>>
>>>>>>>> implies that the `joinedTable` has the same key as the left input
>>>>>>>> table.
>>>>>>>> IMHO, this does not work because if table2 contains multiple rows
>>>>>>>> that
>>>>>>>> join with a record in table1 (what is the main purpose of a foreign
>>>>>>>> key
>>>>>>>> join), the result table would only contain a single join result, but
>>>>>>>> not
>>>>>>>> multiple.
>>>>>>>>
>>>>>>>> Example:
>>>>>>>>
>>>>>>>> table1 input stream: <A,X>
>>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>>>
>>>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If
>>>>>>>> the
>>>>>>>> result key is the same key as key of table1, this implies that the
>>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>>>>>>> Because the share the same key, whatever result record we emit later,
>>>>>>>> overwrite the previous result.
>>>>>>>>
>>>>>>>> This is the reason why Jan originally proposed to use a combination
>>>>>>>> of
>>>>>>>> both primary keys of the input tables as key of the output table.
>>>>>>>> This
>>>>>>>> makes the keys of the output table unique and we can store both in
>>>>>>>> the
>>>>>>>> output table:
>>>>>>>>
>>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>>>>
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>>
>>>>>>>> Just on remark here.
>>>>>>>>> The high-watermark could be disregarded. The decision about the
>>>>>>>>> forward
>>>>>>>>> depends on the size of the aggregated map.
>>>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element
>>>>>>>>> maps
>>>>>>>>> would be published as delete. Any other count
>>>>>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>>>>>>
>>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>>>
>>>>>>>>> It does look like I could replace the second repartition store and
>>>>>>>>>> highwater store with a groupBy and reduce.  However, it looks like
>>>>>>>>>> I
>>>>>>>>>> would
>>>>>>>>>> still need to store the highwater value within the materialized
>>>>>>>>>> store,
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>> compare the arrival of out-of-order records (assuming my
>>>>>>>>> understanding
>>>>>>>>> of
>>>>>>>>> THIS is correct...). This in effect is the same as the design I have
>>>>>>>>> now,
>>>>>>>>> just with the two tables merged together.
>>>>>>>>>


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

Could you elaborate how would the hwm be defined for each record, and the
detailed logic of Highwater Mark Processor (e.g. how to bound the space
usage of the Highwater Mark store? would it be growing indefinitly)? Maybe
it's already included in your running PR but I cannot fully digest it
completely.


Guozhang


On Sat, Sep 8, 2018 at 7:00 AM, Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Jan
>
> I have included a diagram of what I attempted on the KIP.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 213+Support+non-key+joining+in+KTable#KIP-213Supportnon-
> keyjoininginKTable-GroupBy+Reduce/Aggregate
>
> I attempted this back at the start of my own implementation of this
> solution, and since I could not get it to work I have since discarded the
> code. At this point in time, if you wish to continue pursuing for your
> groupBy solution, I ask that you please create a diagram on the KIP
> carefully explaining your solution. Please feel free to use the image I
> just posted as a starting point. I am having trouble understanding your
> explanations but I think that a carefully constructed diagram will clear up
> any misunderstandings. Alternately, please post a comprehensive PR with
> your solution. I can only guess at what you mean, and since I value my own
> time as much as you value yours, I believe it is your responsibility to
> provide an implementation instead of me trying to guess.
>
> Adam
>
>
>
>
>
>
>
>
>
> On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > Hi James,
> >
> > nice to see you beeing interested. kafka streams at this point supports
> > all sorts of joins as long as both streams have the same key.
> > Adam is currently implementing a join where a KTable and a KTable can
> have
> > a one to many relation ship (1:n). We exploit that rocksdb is a
> > datastore that keeps data sorted (At least exposes an API to access the
> > stored data in a sorted fashion).
> >
> > I think the technical caveats are well understood now and we are
> basically
> > down to philosophy and API Design ( when Adam sees my newest message).
> > I have a lengthy track record of loosing those kinda arguments within the
> > streams community and I have no clue why. So I literally can't wait for
> you
> > to churn through this thread and give you opinion on how we should design
> > the return type of the oneToManyJoin and how many power we want to give
> to
> > the user vs "simplicity" (where simplicity isn't really that as users
> still
> > need to understand it I argue)
> >
> > waiting for you to join in on the discussion
> >
> > Best Jan
> >
> >
> >
> > On 07.09.2018 15:49, James Kwan wrote:
> >
> >> I am new to this group and I found this subject interesting.  Sounds
> like
> >> you guys want to implement a join table of two streams? Is there
> somewhere
> >> I can see the original requirement or proposal?
> >>
> >> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
> >>> wrote:
> >>>
> >>>
> >>> On 05.09.2018 22:17, Adam Bellemare wrote:
> >>>
> >>>> I'm currently testing using a Windowed Store to store the highwater
> >>>> mark.
> >>>> By all indications this should work fine, with the caveat being that
> it
> >>>> can
> >>>> only resolve out-of-order arrival for up to the size of the window
> (ie:
> >>>> 24h, 72h, etc). This would remove the possibility of it being
> unbounded
> >>>> in
> >>>> size.
> >>>>
> >>>> With regards to Jan's suggestion, I believe this is where we will have
> >>>> to
> >>>> remain in disagreement. While I do not disagree with your statement
> >>>> about
> >>>> there likely to be additional joins done in a real-world workflow, I
> do
> >>>> not
> >>>> see how you can conclusively deal with out-of-order arrival of
> >>>> foreign-key
> >>>> changes and subsequent joins. I have attempted what I think you have
> >>>> proposed (without a high-water, using groupBy and reduce) and found
> >>>> that if
> >>>> the foreign key changes too quickly, or the load on a stream thread is
> >>>> too
> >>>> high, the joined messages will arrive out-of-order and be incorrectly
> >>>> propagated, such that an intermediate event is represented as the
> final
> >>>> event.
> >>>>
> >>> Can you shed some light on your groupBy implementation. There must be
> >>> some sort of flaw in it.
> >>> I have a suspicion where it is, I would just like to confirm. The idea
> >>> is bullet proof and it must be
> >>> an implementation mess up. I would like to clarify before we draw a
> >>> conclusion.
> >>>
> >>>   Repartitioning the scattered events back to their original
> >>>> partitions is the only way I know how to conclusively deal with
> >>>> out-of-order events in a given time frame, and to ensure that the data
> >>>> is
> >>>> eventually consistent with the input events.
> >>>>
> >>>> If you have some code to share that illustrates your approach, I would
> >>>> be
> >>>> very grateful as it would remove any misunderstandings that I may
> have.
> >>>>
> >>> ah okay you were looking for my code. I don't have something easily
> >>> readable here as its bloated with OO-patterns.
> >>>
> >>> its anyhow trivial:
> >>>
> >>> @Override
> >>>     public T apply(K aggKey, V value, T aggregate)
> >>>     {
> >>>         Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
> >>>         U toModifyKey = mapper.apply(value);
> >>>             << this is the place where people actually gonna have
> issues
> >>> and why you probably couldn't do it. we would need to find a solution
> here.
> >>> I didn't realize that yet.
> >>>             << we propagate the field in the joiner, so that we can
> pick
> >>> it up in an aggregate. Probably you have not thought of this in your
> >>> approach right?
> >>>             << I am very open to find a generic solution here. In my
> >>> honest opinion this is broken in KTableImpl.GroupBy that it looses the
> keys
> >>> and only maintains the aggregate key.
> >>>             << I abstracted it away back then way before i was thinking
> >>> of oneToMany join. That is why I didn't realize its significance here.
> >>>             << Opinions?
> >>>
> >>>         for (V m : current)
> >>>         {
> >>>             currentStateAsMap.put(mapper.apply(m), m);
> >>>         }
> >>>         if (isAdder)
> >>>         {
> >>>             currentStateAsMap.put(toModifyKey, value);
> >>>         }
> >>>         else
> >>>         {
> >>>             currentStateAsMap.remove(toModifyKey);
> >>>             if(currentStateAsMap.isEmpty()){
> >>>                 return null;
> >>>             }
> >>>         }
> >>>         retrun asAggregateType(currentStateAsMap)
> >>>     }
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> Thanks,
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <
> Jan.Filipiak@trivago.com>
> >>>> wrote:
> >>>>
> >>>> Thanks Adam for bringing Matthias to speed!
> >>>>>
> >>>>> about the differences. I think re-keying back should be optional at
> >>>>> best.
> >>>>> I would say we return a KScatteredTable with reshuffle() returning
> >>>>> KTable<originalKey,Joined> to make the backwards repartitioning
> >>>>> optional.
> >>>>> I am also in a big favour of doing the out of order processing using
> >>>>> group
> >>>>> by instead high water mark tracking.
> >>>>> Just because unbounded growth is just scary + It saves us the header
> >>>>> stuff.
> >>>>>
> >>>>> I think the abstraction of always repartitioning back is just not so
> >>>>> strong. Like the work has been done before we partition back and
> >>>>> grouping
> >>>>> by something else afterwards is really common.
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>>>
> >>>>> Hi Matthias
> >>>>>>
> >>>>>> Thank you for your feedback, I do appreciate it!
> >>>>>>
> >>>>>> While name spacing would be possible, it would require to
> deserialize
> >>>>>>
> >>>>>>> user headers what implies a runtime overhead. I would suggest to no
> >>>>>>> namespace for now to avoid the overhead. If this becomes a problem
> in
> >>>>>>> the future, we can still add name spacing later on.
> >>>>>>>
> >>>>>>> Agreed. I will go with using a reserved string and document it.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> My main concern about the design it the type of the result KTable:
> If
> >>>>>> I
> >>>>>> understood the proposal correctly,
> >>>>>>
> >>>>>>
> >>>>>> In your example, you have table1 and table2 swapped. Here is how it
> >>>>>> works
> >>>>>> currently:
> >>>>>>
> >>>>>> 1) table1 has the records that contain the foreign key within their
> >>>>>> value.
> >>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
> >>>>>> <c,(fk=B,bar=3)>
> >>>>>> table2 input stream: <A,X>, <B,Y>
> >>>>>>
> >>>>>> 2) A Value mapper is required to extract the foreign key.
> >>>>>> table1 foreign key mapper: ( value => value.fk )
> >>>>>>
> >>>>>> The mapper is applied to each element in table1, and a new combined
> >>>>>> key is
> >>>>>> made:
> >>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
> >>>>>> (fk=B,bar=3)>
> >>>>>>
> >>>>>> 3) The rekeyed events are copartitioned with table2:
> >>>>>>
> >>>>>> a) Stream Thread with Partition 0:
> >>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
> >>>>>> Table2: <A,X>
> >>>>>>
> >>>>>> b) Stream Thread with Partition 1:
> >>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>>>> Table2: <B,Y>
> >>>>>>
> >>>>>> 4) From here, they can be joined together locally by applying the
> >>>>>> joiner
> >>>>>> function.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> At this point, Jan's design and my design deviate. My design goes on
> >>>>>> to
> >>>>>> repartition the data post-join and resolve out-of-order arrival of
> >>>>>> records,
> >>>>>> finally returning the data keyed just the original key. I do not
> >>>>>> expose
> >>>>>> the
> >>>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
> >>>>>> function. This does make for larger footprint, but it removes all
> >>>>>> agency
> >>>>>> for resolving out-of-order arrivals and handling CombinedKeys from
> the
> >>>>>> user. I believe that this makes the function much easier to use.
> >>>>>>
> >>>>>> Let me know if this helps resolve your questions, and please feel
> >>>>>> free to
> >>>>>> add anything else on your mind.
> >>>>>>
> >>>>>> Thanks again,
> >>>>>> Adam
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> >>>>>> matthias@confluent.io>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi,
> >>>>>>
> >>>>>>> I am just catching up on this thread. I did not read everything so
> >>>>>>> far,
> >>>>>>> but want to share couple of initial thoughts:
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> Headers: I think there is a fundamental difference between header
> >>>>>>> usage
> >>>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic
> >>>>>>> that
> >>>>>>> are owned by Kafka Streams and nobody else is supposed to write
> into
> >>>>>>> them. In fact, no user header are written into the changelog topic
> >>>>>>> and
> >>>>>>> thus, there are not conflicts.
> >>>>>>>
> >>>>>>> Nevertheless, I don't see a big issue with using headers within
> >>>>>>> Streams.
> >>>>>>> As long as we document it, we can have some "reserved" header keys
> >>>>>>> and
> >>>>>>> users are not allowed to use when processing data with Kafka
> Streams.
> >>>>>>> IMHO, this should be ok.
> >>>>>>>
> >>>>>>> I think there is a safe way to avoid conflicts, since these headers
> >>>>>>> are
> >>>>>>>
> >>>>>>>> only needed in internal topics (I think):
> >>>>>>>> For internal and changelog topics, we can namespace all headers:
> >>>>>>>> * user-defined headers are namespaced as "external." + headerKey
> >>>>>>>> * internal headers are namespaced as "internal." + headerKey
> >>>>>>>>
> >>>>>>>> While name spacing would be possible, it would require to
> >>>>>>> deserialize
> >>>>>>> user headers what implies a runtime overhead. I would suggest to no
> >>>>>>> namespace for now to avoid the overhead. If this becomes a problem
> in
> >>>>>>> the future, we can still add name spacing later on.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> My main concern about the design it the type of the result KTable:
> >>>>>>> If I
> >>>>>>> understood the proposal correctly,
> >>>>>>>
> >>>>>>> KTable<K1,V1> table1 = ...
> >>>>>>> KTable<K2,V2> table2 = ...
> >>>>>>>
> >>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
> >>>>>>>
> >>>>>>> implies that the `joinedTable` has the same key as the left input
> >>>>>>> table.
> >>>>>>> IMHO, this does not work because if table2 contains multiple rows
> >>>>>>> that
> >>>>>>> join with a record in table1 (what is the main purpose of a foreign
> >>>>>>> key
> >>>>>>> join), the result table would only contain a single join result,
> but
> >>>>>>> not
> >>>>>>> multiple.
> >>>>>>>
> >>>>>>> Example:
> >>>>>>>
> >>>>>>> table1 input stream: <A,X>
> >>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>>>
> >>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If
> >>>>>>> the
> >>>>>>> result key is the same key as key of table1, this implies that the
> >>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
> >>>>>>> Because the share the same key, whatever result record we emit
> later,
> >>>>>>> overwrite the previous result.
> >>>>>>>
> >>>>>>> This is the reason why Jan originally proposed to use a combination
> >>>>>>> of
> >>>>>>> both primary keys of the input tables as key of the output table.
> >>>>>>> This
> >>>>>>> makes the keys of the output table unique and we can store both in
> >>>>>>> the
> >>>>>>> output table:
> >>>>>>>
> >>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> >>>>>>>
> >>>>>>>
> >>>>>>> Thoughts?
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>>>
> >>>>>>> Just on remark here.
> >>>>>>>> The high-watermark could be disregarded. The decision about the
> >>>>>>>> forward
> >>>>>>>> depends on the size of the aggregated map.
> >>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0
> element
> >>>>>>>> maps
> >>>>>>>> would be published as delete. Any other count
> >>>>>>>> of map entries is in "waiting for correct deletes to
> arrive"-state.
> >>>>>>>>
> >>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>>>
> >>>>>>>> It does look like I could replace the second repartition store and
> >>>>>>>>> highwater store with a groupBy and reduce.  However, it looks
> like
> >>>>>>>>> I
> >>>>>>>>> would
> >>>>>>>>> still need to store the highwater value within the materialized
> >>>>>>>>> store,
> >>>>>>>>>
> >>>>>>>>> to
> >>>>>>>> compare the arrival of out-of-order records (assuming my
> >>>>>>>> understanding
> >>>>>>>> of
> >>>>>>>> THIS is correct...). This in effect is the same as the design I
> have
> >>>>>>>> now,
> >>>>>>>> just with the two tables merged together.
> >>>>>>>>
> >>>>>>>
> >
>



-- 
-- Guozhang

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Jan

I have included a diagram of what I attempted on the KIP.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-GroupBy+Reduce/Aggregate

I attempted this back at the start of my own implementation of this
solution, and since I could not get it to work I have since discarded the
code. At this point in time, if you wish to continue pursuing for your
groupBy solution, I ask that you please create a diagram on the KIP
carefully explaining your solution. Please feel free to use the image I
just posted as a starting point. I am having trouble understanding your
explanations but I think that a carefully constructed diagram will clear up
any misunderstandings. Alternately, please post a comprehensive PR with
your solution. I can only guess at what you mean, and since I value my own
time as much as you value yours, I believe it is your responsibility to
provide an implementation instead of me trying to guess.

Adam









On Sat, Sep 8, 2018 at 8:00 AM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi James,
>
> nice to see you beeing interested. kafka streams at this point supports
> all sorts of joins as long as both streams have the same key.
> Adam is currently implementing a join where a KTable and a KTable can have
> a one to many relation ship (1:n). We exploit that rocksdb is a
> datastore that keeps data sorted (At least exposes an API to access the
> stored data in a sorted fashion).
>
> I think the technical caveats are well understood now and we are basically
> down to philosophy and API Design ( when Adam sees my newest message).
> I have a lengthy track record of loosing those kinda arguments within the
> streams community and I have no clue why. So I literally can't wait for you
> to churn through this thread and give you opinion on how we should design
> the return type of the oneToManyJoin and how many power we want to give to
> the user vs "simplicity" (where simplicity isn't really that as users still
> need to understand it I argue)
>
> waiting for you to join in on the discussion
>
> Best Jan
>
>
>
> On 07.09.2018 15:49, James Kwan wrote:
>
>> I am new to this group and I found this subject interesting.  Sounds like
>> you guys want to implement a join table of two streams? Is there somewhere
>> I can see the original requirement or proposal?
>>
>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>
>>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>>
>>>> I'm currently testing using a Windowed Store to store the highwater
>>>> mark.
>>>> By all indications this should work fine, with the caveat being that it
>>>> can
>>>> only resolve out-of-order arrival for up to the size of the window (ie:
>>>> 24h, 72h, etc). This would remove the possibility of it being unbounded
>>>> in
>>>> size.
>>>>
>>>> With regards to Jan's suggestion, I believe this is where we will have
>>>> to
>>>> remain in disagreement. While I do not disagree with your statement
>>>> about
>>>> there likely to be additional joins done in a real-world workflow, I do
>>>> not
>>>> see how you can conclusively deal with out-of-order arrival of
>>>> foreign-key
>>>> changes and subsequent joins. I have attempted what I think you have
>>>> proposed (without a high-water, using groupBy and reduce) and found
>>>> that if
>>>> the foreign key changes too quickly, or the load on a stream thread is
>>>> too
>>>> high, the joined messages will arrive out-of-order and be incorrectly
>>>> propagated, such that an intermediate event is represented as the final
>>>> event.
>>>>
>>> Can you shed some light on your groupBy implementation. There must be
>>> some sort of flaw in it.
>>> I have a suspicion where it is, I would just like to confirm. The idea
>>> is bullet proof and it must be
>>> an implementation mess up. I would like to clarify before we draw a
>>> conclusion.
>>>
>>>   Repartitioning the scattered events back to their original
>>>> partitions is the only way I know how to conclusively deal with
>>>> out-of-order events in a given time frame, and to ensure that the data
>>>> is
>>>> eventually consistent with the input events.
>>>>
>>>> If you have some code to share that illustrates your approach, I would
>>>> be
>>>> very grateful as it would remove any misunderstandings that I may have.
>>>>
>>> ah okay you were looking for my code. I don't have something easily
>>> readable here as its bloated with OO-patterns.
>>>
>>> its anyhow trivial:
>>>
>>> @Override
>>>     public T apply(K aggKey, V value, T aggregate)
>>>     {
>>>         Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
>>>         U toModifyKey = mapper.apply(value);
>>>             << this is the place where people actually gonna have issues
>>> and why you probably couldn't do it. we would need to find a solution here.
>>> I didn't realize that yet.
>>>             << we propagate the field in the joiner, so that we can pick
>>> it up in an aggregate. Probably you have not thought of this in your
>>> approach right?
>>>             << I am very open to find a generic solution here. In my
>>> honest opinion this is broken in KTableImpl.GroupBy that it looses the keys
>>> and only maintains the aggregate key.
>>>             << I abstracted it away back then way before i was thinking
>>> of oneToMany join. That is why I didn't realize its significance here.
>>>             << Opinions?
>>>
>>>         for (V m : current)
>>>         {
>>>             currentStateAsMap.put(mapper.apply(m), m);
>>>         }
>>>         if (isAdder)
>>>         {
>>>             currentStateAsMap.put(toModifyKey, value);
>>>         }
>>>         else
>>>         {
>>>             currentStateAsMap.remove(toModifyKey);
>>>             if(currentStateAsMap.isEmpty()){
>>>                 return null;
>>>             }
>>>         }
>>>         retrun asAggregateType(currentStateAsMap)
>>>     }
>>>
>>>
>>>
>>>
>>>
>>> Thanks,
>>>>
>>>> Adam
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>> Thanks Adam for bringing Matthias to speed!
>>>>>
>>>>> about the differences. I think re-keying back should be optional at
>>>>> best.
>>>>> I would say we return a KScatteredTable with reshuffle() returning
>>>>> KTable<originalKey,Joined> to make the backwards repartitioning
>>>>> optional.
>>>>> I am also in a big favour of doing the out of order processing using
>>>>> group
>>>>> by instead high water mark tracking.
>>>>> Just because unbounded growth is just scary + It saves us the header
>>>>> stuff.
>>>>>
>>>>> I think the abstraction of always repartitioning back is just not so
>>>>> strong. Like the work has been done before we partition back and
>>>>> grouping
>>>>> by something else afterwards is really common.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>>
>>>>> Hi Matthias
>>>>>>
>>>>>> Thank you for your feedback, I do appreciate it!
>>>>>>
>>>>>> While name spacing would be possible, it would require to deserialize
>>>>>>
>>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>>> the future, we can still add name spacing later on.
>>>>>>>
>>>>>>> Agreed. I will go with using a reserved string and document it.
>>>>>>
>>>>>>
>>>>>>
>>>>>> My main concern about the design it the type of the result KTable: If
>>>>>> I
>>>>>> understood the proposal correctly,
>>>>>>
>>>>>>
>>>>>> In your example, you have table1 and table2 swapped. Here is how it
>>>>>> works
>>>>>> currently:
>>>>>>
>>>>>> 1) table1 has the records that contain the foreign key within their
>>>>>> value.
>>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
>>>>>> <c,(fk=B,bar=3)>
>>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>>
>>>>>> 2) A Value mapper is required to extract the foreign key.
>>>>>> table1 foreign key mapper: ( value => value.fk )
>>>>>>
>>>>>> The mapper is applied to each element in table1, and a new combined
>>>>>> key is
>>>>>> made:
>>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>>>>>> (fk=B,bar=3)>
>>>>>>
>>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>>
>>>>>> a) Stream Thread with Partition 0:
>>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>>>>>> Table2: <A,X>
>>>>>>
>>>>>> b) Stream Thread with Partition 1:
>>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>>> Table2: <B,Y>
>>>>>>
>>>>>> 4) From here, they can be joined together locally by applying the
>>>>>> joiner
>>>>>> function.
>>>>>>
>>>>>>
>>>>>>
>>>>>> At this point, Jan's design and my design deviate. My design goes on
>>>>>> to
>>>>>> repartition the data post-join and resolve out-of-order arrival of
>>>>>> records,
>>>>>> finally returning the data keyed just the original key. I do not
>>>>>> expose
>>>>>> the
>>>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
>>>>>> function. This does make for larger footprint, but it removes all
>>>>>> agency
>>>>>> for resolving out-of-order arrivals and handling CombinedKeys from the
>>>>>> user. I believe that this makes the function much easier to use.
>>>>>>
>>>>>> Let me know if this helps resolve your questions, and please feel
>>>>>> free to
>>>>>> add anything else on your mind.
>>>>>>
>>>>>> Thanks again,
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
>>>>>> matthias@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>> Hi,
>>>>>>
>>>>>>> I am just catching up on this thread. I did not read everything so
>>>>>>> far,
>>>>>>> but want to share couple of initial thoughts:
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Headers: I think there is a fundamental difference between header
>>>>>>> usage
>>>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic
>>>>>>> that
>>>>>>> are owned by Kafka Streams and nobody else is supposed to write into
>>>>>>> them. In fact, no user header are written into the changelog topic
>>>>>>> and
>>>>>>> thus, there are not conflicts.
>>>>>>>
>>>>>>> Nevertheless, I don't see a big issue with using headers within
>>>>>>> Streams.
>>>>>>> As long as we document it, we can have some "reserved" header keys
>>>>>>> and
>>>>>>> users are not allowed to use when processing data with Kafka Streams.
>>>>>>> IMHO, this should be ok.
>>>>>>>
>>>>>>> I think there is a safe way to avoid conflicts, since these headers
>>>>>>> are
>>>>>>>
>>>>>>>> only needed in internal topics (I think):
>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>
>>>>>>>> While name spacing would be possible, it would require to
>>>>>>> deserialize
>>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>>> the future, we can still add name spacing later on.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> My main concern about the design it the type of the result KTable:
>>>>>>> If I
>>>>>>> understood the proposal correctly,
>>>>>>>
>>>>>>> KTable<K1,V1> table1 = ...
>>>>>>> KTable<K2,V2> table2 = ...
>>>>>>>
>>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>>>
>>>>>>> implies that the `joinedTable` has the same key as the left input
>>>>>>> table.
>>>>>>> IMHO, this does not work because if table2 contains multiple rows
>>>>>>> that
>>>>>>> join with a record in table1 (what is the main purpose of a foreign
>>>>>>> key
>>>>>>> join), the result table would only contain a single join result, but
>>>>>>> not
>>>>>>> multiple.
>>>>>>>
>>>>>>> Example:
>>>>>>>
>>>>>>> table1 input stream: <A,X>
>>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>>
>>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If
>>>>>>> the
>>>>>>> result key is the same key as key of table1, this implies that the
>>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>>>>>> Because the share the same key, whatever result record we emit later,
>>>>>>> overwrite the previous result.
>>>>>>>
>>>>>>> This is the reason why Jan originally proposed to use a combination
>>>>>>> of
>>>>>>> both primary keys of the input tables as key of the output table.
>>>>>>> This
>>>>>>> makes the keys of the output table unique and we can store both in
>>>>>>> the
>>>>>>> output table:
>>>>>>>
>>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>>>
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>>
>>>>>>> Just on remark here.
>>>>>>>> The high-watermark could be disregarded. The decision about the
>>>>>>>> forward
>>>>>>>> depends on the size of the aggregated map.
>>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element
>>>>>>>> maps
>>>>>>>> would be published as delete. Any other count
>>>>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>>>>>
>>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>>
>>>>>>>> It does look like I could replace the second repartition store and
>>>>>>>>> highwater store with a groupBy and reduce.  However, it looks like
>>>>>>>>> I
>>>>>>>>> would
>>>>>>>>> still need to store the highwater value within the materialized
>>>>>>>>> store,
>>>>>>>>>
>>>>>>>>> to
>>>>>>>> compare the arrival of out-of-order records (assuming my
>>>>>>>> understanding
>>>>>>>> of
>>>>>>>> THIS is correct...). This in effect is the same as the design I have
>>>>>>>> now,
>>>>>>>> just with the two tables merged together.
>>>>>>>>
>>>>>>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

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

nice to see you beeing interested. kafka streams at this point supports 
all sorts of joins as long as both streams have the same key.
Adam is currently implementing a join where a KTable and a KTable can 
have a one to many relation ship (1:n). We exploit that rocksdb is a
datastore that keeps data sorted (At least exposes an API to access the 
stored data in a sorted fashion).

I think the technical caveats are well understood now and we are 
basically down to philosophy and API Design ( when Adam sees my newest 
message).
I have a lengthy track record of loosing those kinda arguments within 
the streams community and I have no clue why. So I literally can't wait 
for you to churn through this thread and give you opinion on how we 
should design the return type of the oneToManyJoin and how many power we 
want to give to the user vs "simplicity" (where simplicity isn't really 
that as users still need to understand it I argue)

waiting for you to join in on the discussion

Best Jan


On 07.09.2018 15:49, James Kwan wrote:
> I am new to this group and I found this subject interesting.  Sounds like you guys want to implement a join table of two streams? Is there somewhere I can see the original requirement or proposal?
>
>> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com> wrote:
>>
>>
>> On 05.09.2018 22:17, Adam Bellemare wrote:
>>> I'm currently testing using a Windowed Store to store the highwater mark.
>>> By all indications this should work fine, with the caveat being that it can
>>> only resolve out-of-order arrival for up to the size of the window (ie:
>>> 24h, 72h, etc). This would remove the possibility of it being unbounded in
>>> size.
>>>
>>> With regards to Jan's suggestion, I believe this is where we will have to
>>> remain in disagreement. While I do not disagree with your statement about
>>> there likely to be additional joins done in a real-world workflow, I do not
>>> see how you can conclusively deal with out-of-order arrival of foreign-key
>>> changes and subsequent joins. I have attempted what I think you have
>>> proposed (without a high-water, using groupBy and reduce) and found that if
>>> the foreign key changes too quickly, or the load on a stream thread is too
>>> high, the joined messages will arrive out-of-order and be incorrectly
>>> propagated, such that an intermediate event is represented as the final
>>> event.
>> Can you shed some light on your groupBy implementation. There must be some sort of flaw in it.
>> I have a suspicion where it is, I would just like to confirm. The idea is bullet proof and it must be
>> an implementation mess up. I would like to clarify before we draw a conclusion.
>>
>>>   Repartitioning the scattered events back to their original
>>> partitions is the only way I know how to conclusively deal with
>>> out-of-order events in a given time frame, and to ensure that the data is
>>> eventually consistent with the input events.
>>>
>>> If you have some code to share that illustrates your approach, I would be
>>> very grateful as it would remove any misunderstandings that I may have.
>> ah okay you were looking for my code. I don't have something easily readable here as its bloated with OO-patterns.
>>
>> its anyhow trivial:
>>
>> @Override
>>     public T apply(K aggKey, V value, T aggregate)
>>     {
>>         Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
>>         U toModifyKey = mapper.apply(value);
>>             << this is the place where people actually gonna have issues and why you probably couldn't do it. we would need to find a solution here. I didn't realize that yet.
>>             << we propagate the field in the joiner, so that we can pick it up in an aggregate. Probably you have not thought of this in your approach right?
>>             << I am very open to find a generic solution here. In my honest opinion this is broken in KTableImpl.GroupBy that it looses the keys and only maintains the aggregate key.
>>             << I abstracted it away back then way before i was thinking of oneToMany join. That is why I didn't realize its significance here.
>>             << Opinions?
>>
>>         for (V m : current)
>>         {
>>             currentStateAsMap.put(mapper.apply(m), m);
>>         }
>>         if (isAdder)
>>         {
>>             currentStateAsMap.put(toModifyKey, value);
>>         }
>>         else
>>         {
>>             currentStateAsMap.remove(toModifyKey);
>>             if(currentStateAsMap.isEmpty()){
>>                 return null;
>>             }
>>         }
>>         retrun asAggregateType(currentStateAsMap)
>>     }
>>
>>
>>
>>
>>
>>> Thanks,
>>>
>>> Adam
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> Thanks Adam for bringing Matthias to speed!
>>>>
>>>> about the differences. I think re-keying back should be optional at best.
>>>> I would say we return a KScatteredTable with reshuffle() returning
>>>> KTable<originalKey,Joined> to make the backwards repartitioning optional.
>>>> I am also in a big favour of doing the out of order processing using group
>>>> by instead high water mark tracking.
>>>> Just because unbounded growth is just scary + It saves us the header stuff.
>>>>
>>>> I think the abstraction of always repartitioning back is just not so
>>>> strong. Like the work has been done before we partition back and grouping
>>>> by something else afterwards is really common.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>>>
>>>>> Hi Matthias
>>>>>
>>>>> Thank you for your feedback, I do appreciate it!
>>>>>
>>>>> While name spacing would be possible, it would require to deserialize
>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>> the future, we can still add name spacing later on.
>>>>>>
>>>>> Agreed. I will go with using a reserved string and document it.
>>>>>
>>>>>
>>>>>
>>>>> My main concern about the design it the type of the result KTable: If I
>>>>> understood the proposal correctly,
>>>>>
>>>>>
>>>>> In your example, you have table1 and table2 swapped. Here is how it works
>>>>> currently:
>>>>>
>>>>> 1) table1 has the records that contain the foreign key within their value.
>>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
>>>>> table2 input stream: <A,X>, <B,Y>
>>>>>
>>>>> 2) A Value mapper is required to extract the foreign key.
>>>>> table1 foreign key mapper: ( value => value.fk )
>>>>>
>>>>> The mapper is applied to each element in table1, and a new combined key is
>>>>> made:
>>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>>>>> (fk=B,bar=3)>
>>>>>
>>>>> 3) The rekeyed events are copartitioned with table2:
>>>>>
>>>>> a) Stream Thread with Partition 0:
>>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>>>>> Table2: <A,X>
>>>>>
>>>>> b) Stream Thread with Partition 1:
>>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>>> Table2: <B,Y>
>>>>>
>>>>> 4) From here, they can be joined together locally by applying the joiner
>>>>> function.
>>>>>
>>>>>
>>>>>
>>>>> At this point, Jan's design and my design deviate. My design goes on to
>>>>> repartition the data post-join and resolve out-of-order arrival of
>>>>> records,
>>>>> finally returning the data keyed just the original key. I do not expose
>>>>> the
>>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
>>>>> function. This does make for larger footprint, but it removes all agency
>>>>> for resolving out-of-order arrivals and handling CombinedKeys from the
>>>>> user. I believe that this makes the function much easier to use.
>>>>>
>>>>> Let me know if this helps resolve your questions, and please feel free to
>>>>> add anything else on your mind.
>>>>>
>>>>> Thanks again,
>>>>> Adam
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>
>>>>> Hi,
>>>>>> I am just catching up on this thread. I did not read everything so far,
>>>>>> but want to share couple of initial thoughts:
>>>>>>
>>>>>>
>>>>>>
>>>>>> Headers: I think there is a fundamental difference between header usage
>>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic that
>>>>>> are owned by Kafka Streams and nobody else is supposed to write into
>>>>>> them. In fact, no user header are written into the changelog topic and
>>>>>> thus, there are not conflicts.
>>>>>>
>>>>>> Nevertheless, I don't see a big issue with using headers within Streams.
>>>>>> As long as we document it, we can have some "reserved" header keys and
>>>>>> users are not allowed to use when processing data with Kafka Streams.
>>>>>> IMHO, this should be ok.
>>>>>>
>>>>>> I think there is a safe way to avoid conflicts, since these headers are
>>>>>>> only needed in internal topics (I think):
>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>
>>>>>> While name spacing would be possible, it would require to deserialize
>>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>>> the future, we can still add name spacing later on.
>>>>>>
>>>>>>
>>>>>>
>>>>>> My main concern about the design it the type of the result KTable: If I
>>>>>> understood the proposal correctly,
>>>>>>
>>>>>> KTable<K1,V1> table1 = ...
>>>>>> KTable<K2,V2> table2 = ...
>>>>>>
>>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>>>
>>>>>> implies that the `joinedTable` has the same key as the left input table.
>>>>>> IMHO, this does not work because if table2 contains multiple rows that
>>>>>> join with a record in table1 (what is the main purpose of a foreign key
>>>>>> join), the result table would only contain a single join result, but not
>>>>>> multiple.
>>>>>>
>>>>>> Example:
>>>>>>
>>>>>> table1 input stream: <A,X>
>>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>>>
>>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
>>>>>> result key is the same key as key of table1, this implies that the
>>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>>>>> Because the share the same key, whatever result record we emit later,
>>>>>> overwrite the previous result.
>>>>>>
>>>>>> This is the reason why Jan originally proposed to use a combination of
>>>>>> both primary keys of the input tables as key of the output table. This
>>>>>> makes the keys of the output table unique and we can store both in the
>>>>>> output table:
>>>>>>
>>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>>>
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>>>
>>>>>>> Just on remark here.
>>>>>>> The high-watermark could be disregarded. The decision about the forward
>>>>>>> depends on the size of the aggregated map.
>>>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element maps
>>>>>>> would be published as delete. Any other count
>>>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>>>>
>>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>>>
>>>>>>>> It does look like I could replace the second repartition store and
>>>>>>>> highwater store with a groupBy and reduce.  However, it looks like I
>>>>>>>> would
>>>>>>>> still need to store the highwater value within the materialized store,
>>>>>>>>
>>>>>>> to
>>>>>>> compare the arrival of out-of-order records (assuming my understanding
>>>>>>> of
>>>>>>> THIS is correct...). This in effect is the same as the design I have
>>>>>>> now,
>>>>>>> just with the two tables merged together.


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by John Roesler <jo...@confluent.io>.
Hi James,

The proposal we are discussing is
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable

I'm not sure if it's been updated to reflect current thinking.

-John

On Fri, Sep 7, 2018 at 8:49 AM James Kwan <jw...@gmail.com> wrote:

> I am new to this group and I found this subject interesting.  Sounds like
> you guys want to implement a join table of two streams? Is there somewhere
> I can see the original requirement or proposal?
>
> > On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
> >
> >
> > On 05.09.2018 22:17, Adam Bellemare wrote:
> >> I'm currently testing using a Windowed Store to store the highwater
> mark.
> >> By all indications this should work fine, with the caveat being that it
> can
> >> only resolve out-of-order arrival for up to the size of the window (ie:
> >> 24h, 72h, etc). This would remove the possibility of it being unbounded
> in
> >> size.
> >>
> >> With regards to Jan's suggestion, I believe this is where we will have
> to
> >> remain in disagreement. While I do not disagree with your statement
> about
> >> there likely to be additional joins done in a real-world workflow, I do
> not
> >> see how you can conclusively deal with out-of-order arrival of
> foreign-key
> >> changes and subsequent joins. I have attempted what I think you have
> >> proposed (without a high-water, using groupBy and reduce) and found
> that if
> >> the foreign key changes too quickly, or the load on a stream thread is
> too
> >> high, the joined messages will arrive out-of-order and be incorrectly
> >> propagated, such that an intermediate event is represented as the final
> >> event.
> > Can you shed some light on your groupBy implementation. There must be
> some sort of flaw in it.
> > I have a suspicion where it is, I would just like to confirm. The idea
> is bullet proof and it must be
> > an implementation mess up. I would like to clarify before we draw a
> conclusion.
> >
> >>  Repartitioning the scattered events back to their original
> >> partitions is the only way I know how to conclusively deal with
> >> out-of-order events in a given time frame, and to ensure that the data
> is
> >> eventually consistent with the input events.
> >>
> >> If you have some code to share that illustrates your approach, I would
> be
> >> very grateful as it would remove any misunderstandings that I may have.
> >
> > ah okay you were looking for my code. I don't have something easily
> readable here as its bloated with OO-patterns.
> >
> > its anyhow trivial:
> >
> > @Override
> >    public T apply(K aggKey, V value, T aggregate)
> >    {
> >        Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
> >        U toModifyKey = mapper.apply(value);
> >            << this is the place where people actually gonna have issues
> and why you probably couldn't do it. we would need to find a solution here.
> I didn't realize that yet.
> >            << we propagate the field in the joiner, so that we can pick
> it up in an aggregate. Probably you have not thought of this in your
> approach right?
> >            << I am very open to find a generic solution here. In my
> honest opinion this is broken in KTableImpl.GroupBy that it looses the keys
> and only maintains the aggregate key.
> >            << I abstracted it away back then way before i was thinking
> of oneToMany join. That is why I didn't realize its significance here.
> >            << Opinions?
> >
> >        for (V m : current)
> >        {
> >            currentStateAsMap.put(mapper.apply(m), m);
> >        }
> >        if (isAdder)
> >        {
> >            currentStateAsMap.put(toModifyKey, value);
> >        }
> >        else
> >        {
> >            currentStateAsMap.remove(toModifyKey);
> >            if(currentStateAsMap.isEmpty()){
> >                return null;
> >            }
> >        }
> >        retrun asAggregateType(currentStateAsMap)
> >    }
> >
> >
> >
> >
> >
> >>
> >> Thanks,
> >>
> >> Adam
> >>
> >>
> >>
> >>
> >>
> >> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
> >> wrote:
> >>
> >>> Thanks Adam for bringing Matthias to speed!
> >>>
> >>> about the differences. I think re-keying back should be optional at
> best.
> >>> I would say we return a KScatteredTable with reshuffle() returning
> >>> KTable<originalKey,Joined> to make the backwards repartitioning
> optional.
> >>> I am also in a big favour of doing the out of order processing using
> group
> >>> by instead high water mark tracking.
> >>> Just because unbounded growth is just scary + It saves us the header
> stuff.
> >>>
> >>> I think the abstraction of always repartitioning back is just not so
> >>> strong. Like the work has been done before we partition back and
> grouping
> >>> by something else afterwards is really common.
> >>>
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> On 05.09.2018 13:49, Adam Bellemare wrote:
> >>>
> >>>> Hi Matthias
> >>>>
> >>>> Thank you for your feedback, I do appreciate it!
> >>>>
> >>>> While name spacing would be possible, it would require to deserialize
> >>>>> user headers what implies a runtime overhead. I would suggest to no
> >>>>> namespace for now to avoid the overhead. If this becomes a problem in
> >>>>> the future, we can still add name spacing later on.
> >>>>>
> >>>> Agreed. I will go with using a reserved string and document it.
> >>>>
> >>>>
> >>>>
> >>>> My main concern about the design it the type of the result KTable: If
> I
> >>>> understood the proposal correctly,
> >>>>
> >>>>
> >>>> In your example, you have table1 and table2 swapped. Here is how it
> works
> >>>> currently:
> >>>>
> >>>> 1) table1 has the records that contain the foreign key within their
> value.
> >>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>,
> <c,(fk=B,bar=3)>
> >>>> table2 input stream: <A,X>, <B,Y>
> >>>>
> >>>> 2) A Value mapper is required to extract the foreign key.
> >>>> table1 foreign key mapper: ( value => value.fk )
> >>>>
> >>>> The mapper is applied to each element in table1, and a new combined
> key is
> >>>> made:
> >>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
> >>>> (fk=B,bar=3)>
> >>>>
> >>>> 3) The rekeyed events are copartitioned with table2:
> >>>>
> >>>> a) Stream Thread with Partition 0:
> >>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
> >>>> Table2: <A,X>
> >>>>
> >>>> b) Stream Thread with Partition 1:
> >>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> >>>> Table2: <B,Y>
> >>>>
> >>>> 4) From here, they can be joined together locally by applying the
> joiner
> >>>> function.
> >>>>
> >>>>
> >>>>
> >>>> At this point, Jan's design and my design deviate. My design goes on
> to
> >>>> repartition the data post-join and resolve out-of-order arrival of
> >>>> records,
> >>>> finally returning the data keyed just the original key. I do not
> expose
> >>>> the
> >>>> CombinedKey or any of the internals outside of the joinOnForeignKey
> >>>> function. This does make for larger footprint, but it removes all
> agency
> >>>> for resolving out-of-order arrivals and handling CombinedKeys from the
> >>>> user. I believe that this makes the function much easier to use.
> >>>>
> >>>> Let me know if this helps resolve your questions, and please feel
> free to
> >>>> add anything else on your mind.
> >>>>
> >>>> Thanks again,
> >>>> Adam
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <
> matthias@confluent.io>
> >>>> wrote:
> >>>>
> >>>> Hi,
> >>>>> I am just catching up on this thread. I did not read everything so
> far,
> >>>>> but want to share couple of initial thoughts:
> >>>>>
> >>>>>
> >>>>>
> >>>>> Headers: I think there is a fundamental difference between header
> usage
> >>>>> in this KIP and KP-258. For 258, we add headers to changelog topic
> that
> >>>>> are owned by Kafka Streams and nobody else is supposed to write into
> >>>>> them. In fact, no user header are written into the changelog topic
> and
> >>>>> thus, there are not conflicts.
> >>>>>
> >>>>> Nevertheless, I don't see a big issue with using headers within
> Streams.
> >>>>> As long as we document it, we can have some "reserved" header keys
> and
> >>>>> users are not allowed to use when processing data with Kafka Streams.
> >>>>> IMHO, this should be ok.
> >>>>>
> >>>>> I think there is a safe way to avoid conflicts, since these headers
> are
> >>>>>> only needed in internal topics (I think):
> >>>>>> For internal and changelog topics, we can namespace all headers:
> >>>>>> * user-defined headers are namespaced as "external." + headerKey
> >>>>>> * internal headers are namespaced as "internal." + headerKey
> >>>>>>
> >>>>> While name spacing would be possible, it would require to deserialize
> >>>>> user headers what implies a runtime overhead. I would suggest to no
> >>>>> namespace for now to avoid the overhead. If this becomes a problem in
> >>>>> the future, we can still add name spacing later on.
> >>>>>
> >>>>>
> >>>>>
> >>>>> My main concern about the design it the type of the result KTable:
> If I
> >>>>> understood the proposal correctly,
> >>>>>
> >>>>> KTable<K1,V1> table1 = ...
> >>>>> KTable<K2,V2> table2 = ...
> >>>>>
> >>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
> >>>>>
> >>>>> implies that the `joinedTable` has the same key as the left input
> table.
> >>>>> IMHO, this does not work because if table2 contains multiple rows
> that
> >>>>> join with a record in table1 (what is the main purpose of a foreign
> key
> >>>>> join), the result table would only contain a single join result, but
> not
> >>>>> multiple.
> >>>>>
> >>>>> Example:
> >>>>>
> >>>>> table1 input stream: <A,X>
> >>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
> >>>>>
> >>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If
> the
> >>>>> result key is the same key as key of table1, this implies that the
> >>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
> >>>>> Because the share the same key, whatever result record we emit later,
> >>>>> overwrite the previous result.
> >>>>>
> >>>>> This is the reason why Jan originally proposed to use a combination
> of
> >>>>> both primary keys of the input tables as key of the output table.
> This
> >>>>> makes the keys of the output table unique and we can store both in
> the
> >>>>> output table:
> >>>>>
> >>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
> >>>>>
> >>>>>
> >>>>> Thoughts?
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> >>>>>
> >>>>>> Just on remark here.
> >>>>>> The high-watermark could be disregarded. The decision about the
> forward
> >>>>>> depends on the size of the aggregated map.
> >>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element
> maps
> >>>>>> would be published as delete. Any other count
> >>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
> >>>>>>
> >>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
> >>>>>>
> >>>>>>> It does look like I could replace the second repartition store and
> >>>>>>> highwater store with a groupBy and reduce.  However, it looks like
> I
> >>>>>>> would
> >>>>>>> still need to store the highwater value within the materialized
> store,
> >>>>>>>
> >>>>>> to
> >>>>>> compare the arrival of out-of-order records (assuming my
> understanding
> >>>>>> of
> >>>>>> THIS is correct...). This in effect is the same as the design I have
> >>>>>> now,
> >>>>>> just with the two tables merged together.
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by James Kwan <jw...@gmail.com>.
I am new to this group and I found this subject interesting.  Sounds like you guys want to implement a join table of two streams? Is there somewhere I can see the original requirement or proposal?   

> On Sep 7, 2018, at 8:13 AM, Jan Filipiak <Ja...@trivago.com> wrote:
> 
> 
> On 05.09.2018 22:17, Adam Bellemare wrote:
>> I'm currently testing using a Windowed Store to store the highwater mark.
>> By all indications this should work fine, with the caveat being that it can
>> only resolve out-of-order arrival for up to the size of the window (ie:
>> 24h, 72h, etc). This would remove the possibility of it being unbounded in
>> size.
>> 
>> With regards to Jan's suggestion, I believe this is where we will have to
>> remain in disagreement. While I do not disagree with your statement about
>> there likely to be additional joins done in a real-world workflow, I do not
>> see how you can conclusively deal with out-of-order arrival of foreign-key
>> changes and subsequent joins. I have attempted what I think you have
>> proposed (without a high-water, using groupBy and reduce) and found that if
>> the foreign key changes too quickly, or the load on a stream thread is too
>> high, the joined messages will arrive out-of-order and be incorrectly
>> propagated, such that an intermediate event is represented as the final
>> event.
> Can you shed some light on your groupBy implementation. There must be some sort of flaw in it.
> I have a suspicion where it is, I would just like to confirm. The idea is bullet proof and it must be
> an implementation mess up. I would like to clarify before we draw a conclusion.
> 
>>  Repartitioning the scattered events back to their original
>> partitions is the only way I know how to conclusively deal with
>> out-of-order events in a given time frame, and to ensure that the data is
>> eventually consistent with the input events.
>> 
>> If you have some code to share that illustrates your approach, I would be
>> very grateful as it would remove any misunderstandings that I may have.
> 
> ah okay you were looking for my code. I don't have something easily readable here as its bloated with OO-patterns.
> 
> its anyhow trivial:
> 
> @Override
>    public T apply(K aggKey, V value, T aggregate)
>    {
>        Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
>        U toModifyKey = mapper.apply(value);
>            << this is the place where people actually gonna have issues and why you probably couldn't do it. we would need to find a solution here. I didn't realize that yet.
>            << we propagate the field in the joiner, so that we can pick it up in an aggregate. Probably you have not thought of this in your approach right?
>            << I am very open to find a generic solution here. In my honest opinion this is broken in KTableImpl.GroupBy that it looses the keys and only maintains the aggregate key.
>            << I abstracted it away back then way before i was thinking of oneToMany join. That is why I didn't realize its significance here.
>            << Opinions?
> 
>        for (V m : current)
>        {
>            currentStateAsMap.put(mapper.apply(m), m);
>        }
>        if (isAdder)
>        {
>            currentStateAsMap.put(toModifyKey, value);
>        }
>        else
>        {
>            currentStateAsMap.remove(toModifyKey);
>            if(currentStateAsMap.isEmpty()){
>                return null;
>            }
>        }
>        retrun asAggregateType(currentStateAsMap)
>    }
> 
> 
> 
> 
> 
>> 
>> Thanks,
>> 
>> Adam
>> 
>> 
>> 
>> 
>> 
>> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>> 
>>> Thanks Adam for bringing Matthias to speed!
>>> 
>>> about the differences. I think re-keying back should be optional at best.
>>> I would say we return a KScatteredTable with reshuffle() returning
>>> KTable<originalKey,Joined> to make the backwards repartitioning optional.
>>> I am also in a big favour of doing the out of order processing using group
>>> by instead high water mark tracking.
>>> Just because unbounded growth is just scary + It saves us the header stuff.
>>> 
>>> I think the abstraction of always repartitioning back is just not so
>>> strong. Like the work has been done before we partition back and grouping
>>> by something else afterwards is really common.
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>> 
>>>> Hi Matthias
>>>> 
>>>> Thank you for your feedback, I do appreciate it!
>>>> 
>>>> While name spacing would be possible, it would require to deserialize
>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>> the future, we can still add name spacing later on.
>>>>> 
>>>> Agreed. I will go with using a reserved string and document it.
>>>> 
>>>> 
>>>> 
>>>> My main concern about the design it the type of the result KTable: If I
>>>> understood the proposal correctly,
>>>> 
>>>> 
>>>> In your example, you have table1 and table2 swapped. Here is how it works
>>>> currently:
>>>> 
>>>> 1) table1 has the records that contain the foreign key within their value.
>>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
>>>> table2 input stream: <A,X>, <B,Y>
>>>> 
>>>> 2) A Value mapper is required to extract the foreign key.
>>>> table1 foreign key mapper: ( value => value.fk )
>>>> 
>>>> The mapper is applied to each element in table1, and a new combined key is
>>>> made:
>>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>>>> (fk=B,bar=3)>
>>>> 
>>>> 3) The rekeyed events are copartitioned with table2:
>>>> 
>>>> a) Stream Thread with Partition 0:
>>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>>>> Table2: <A,X>
>>>> 
>>>> b) Stream Thread with Partition 1:
>>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>>> Table2: <B,Y>
>>>> 
>>>> 4) From here, they can be joined together locally by applying the joiner
>>>> function.
>>>> 
>>>> 
>>>> 
>>>> At this point, Jan's design and my design deviate. My design goes on to
>>>> repartition the data post-join and resolve out-of-order arrival of
>>>> records,
>>>> finally returning the data keyed just the original key. I do not expose
>>>> the
>>>> CombinedKey or any of the internals outside of the joinOnForeignKey
>>>> function. This does make for larger footprint, but it removes all agency
>>>> for resolving out-of-order arrivals and handling CombinedKeys from the
>>>> user. I believe that this makes the function much easier to use.
>>>> 
>>>> Let me know if this helps resolve your questions, and please feel free to
>>>> add anything else on your mind.
>>>> 
>>>> Thanks again,
>>>> Adam
>>>> 
>>>> 
>>>> 
>>>> 
>>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>> 
>>>> Hi,
>>>>> I am just catching up on this thread. I did not read everything so far,
>>>>> but want to share couple of initial thoughts:
>>>>> 
>>>>> 
>>>>> 
>>>>> Headers: I think there is a fundamental difference between header usage
>>>>> in this KIP and KP-258. For 258, we add headers to changelog topic that
>>>>> are owned by Kafka Streams and nobody else is supposed to write into
>>>>> them. In fact, no user header are written into the changelog topic and
>>>>> thus, there are not conflicts.
>>>>> 
>>>>> Nevertheless, I don't see a big issue with using headers within Streams.
>>>>> As long as we document it, we can have some "reserved" header keys and
>>>>> users are not allowed to use when processing data with Kafka Streams.
>>>>> IMHO, this should be ok.
>>>>> 
>>>>> I think there is a safe way to avoid conflicts, since these headers are
>>>>>> only needed in internal topics (I think):
>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>> 
>>>>> While name spacing would be possible, it would require to deserialize
>>>>> user headers what implies a runtime overhead. I would suggest to no
>>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>>> the future, we can still add name spacing later on.
>>>>> 
>>>>> 
>>>>> 
>>>>> My main concern about the design it the type of the result KTable: If I
>>>>> understood the proposal correctly,
>>>>> 
>>>>> KTable<K1,V1> table1 = ...
>>>>> KTable<K2,V2> table2 = ...
>>>>> 
>>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>> 
>>>>> implies that the `joinedTable` has the same key as the left input table.
>>>>> IMHO, this does not work because if table2 contains multiple rows that
>>>>> join with a record in table1 (what is the main purpose of a foreign key
>>>>> join), the result table would only contain a single join result, but not
>>>>> multiple.
>>>>> 
>>>>> Example:
>>>>> 
>>>>> table1 input stream: <A,X>
>>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>> 
>>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
>>>>> result key is the same key as key of table1, this implies that the
>>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>>>> Because the share the same key, whatever result record we emit later,
>>>>> overwrite the previous result.
>>>>> 
>>>>> This is the reason why Jan originally proposed to use a combination of
>>>>> both primary keys of the input tables as key of the output table. This
>>>>> makes the keys of the output table unique and we can store both in the
>>>>> output table:
>>>>> 
>>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>> 
>>>>> 
>>>>> Thoughts?
>>>>> 
>>>>> 
>>>>> -Matthias
>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>> 
>>>>>> Just on remark here.
>>>>>> The high-watermark could be disregarded. The decision about the forward
>>>>>> depends on the size of the aggregated map.
>>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element maps
>>>>>> would be published as delete. Any other count
>>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>>> 
>>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>> 
>>>>>>> It does look like I could replace the second repartition store and
>>>>>>> highwater store with a groupBy and reduce.  However, it looks like I
>>>>>>> would
>>>>>>> still need to store the highwater value within the materialized store,
>>>>>>> 
>>>>>> to
>>>>>> compare the arrival of out-of-order records (assuming my understanding
>>>>>> of
>>>>>> THIS is correct...). This in effect is the same as the design I have
>>>>>> now,
>>>>>> just with the two tables merged together.
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
On 05.09.2018 22:17, Adam Bellemare wrote:
> I'm currently testing using a Windowed Store to store the highwater mark.
> By all indications this should work fine, with the caveat being that it can
> only resolve out-of-order arrival for up to the size of the window (ie:
> 24h, 72h, etc). This would remove the possibility of it being unbounded in
> size.
>
> With regards to Jan's suggestion, I believe this is where we will have to
> remain in disagreement. While I do not disagree with your statement about
> there likely to be additional joins done in a real-world workflow, I do not
> see how you can conclusively deal with out-of-order arrival of foreign-key
> changes and subsequent joins. I have attempted what I think you have
> proposed (without a high-water, using groupBy and reduce) and found that if
> the foreign key changes too quickly, or the load on a stream thread is too
> high, the joined messages will arrive out-of-order and be incorrectly
> propagated, such that an intermediate event is represented as the final
> event.
Can you shed some light on your groupBy implementation. There must be 
some sort of flaw in it.
I have a suspicion where it is, I would just like to confirm. The idea 
is bullet proof and it must be
an implementation mess up. I would like to clarify before we draw a 
conclusion.

>   Repartitioning the scattered events back to their original
> partitions is the only way I know how to conclusively deal with
> out-of-order events in a given time frame, and to ensure that the data is
> eventually consistent with the input events.
>
> If you have some code to share that illustrates your approach, I would be
> very grateful as it would remove any misunderstandings that I may have.

ah okay you were looking for my code. I don't have something easily 
readable here as its bloated with OO-patterns.

its anyhow trivial:

@Override
     public T apply(K aggKey, V value, T aggregate)
     {
         Map<U, V> currentStateAsMap = asMap(aggregate); << imaginary
         U toModifyKey = mapper.apply(value);
             << this is the place where people actually gonna have 
issues and why you probably couldn't do it. we would need to find a 
solution here. I didn't realize that yet.
             << we propagate the field in the joiner, so that we can 
pick it up in an aggregate. Probably you have not thought of this in 
your approach right?
             << I am very open to find a generic solution here. In my 
honest opinion this is broken in KTableImpl.GroupBy that it looses the 
keys and only maintains the aggregate key.
             << I abstracted it away back then way before i was thinking 
of oneToMany join. That is why I didn't realize its significance here.
             << Opinions?

         for (V m : current)
         {
             currentStateAsMap.put(mapper.apply(m), m);
         }
         if (isAdder)
         {
             currentStateAsMap.put(toModifyKey, value);
         }
         else
         {
             currentStateAsMap.remove(toModifyKey);
             if(currentStateAsMap.isEmpty()){
                 return null;
             }
         }
         retrun asAggregateType(currentStateAsMap)
     }





>
> Thanks,
>
> Adam
>
>
>
>
>
> On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Thanks Adam for bringing Matthias to speed!
>>
>> about the differences. I think re-keying back should be optional at best.
>> I would say we return a KScatteredTable with reshuffle() returning
>> KTable<originalKey,Joined> to make the backwards repartitioning optional.
>> I am also in a big favour of doing the out of order processing using group
>> by instead high water mark tracking.
>> Just because unbounded growth is just scary + It saves us the header stuff.
>>
>> I think the abstraction of always repartitioning back is just not so
>> strong. Like the work has been done before we partition back and grouping
>> by something else afterwards is really common.
>>
>>
>>
>>
>>
>>
>> On 05.09.2018 13:49, Adam Bellemare wrote:
>>
>>> Hi Matthias
>>>
>>> Thank you for your feedback, I do appreciate it!
>>>
>>> While name spacing would be possible, it would require to deserialize
>>>> user headers what implies a runtime overhead. I would suggest to no
>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>> the future, we can still add name spacing later on.
>>>>
>>> Agreed. I will go with using a reserved string and document it.
>>>
>>>
>>>
>>> My main concern about the design it the type of the result KTable: If I
>>> understood the proposal correctly,
>>>
>>>
>>> In your example, you have table1 and table2 swapped. Here is how it works
>>> currently:
>>>
>>> 1) table1 has the records that contain the foreign key within their value.
>>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
>>> table2 input stream: <A,X>, <B,Y>
>>>
>>> 2) A Value mapper is required to extract the foreign key.
>>> table1 foreign key mapper: ( value => value.fk )
>>>
>>> The mapper is applied to each element in table1, and a new combined key is
>>> made:
>>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>>> (fk=B,bar=3)>
>>>
>>> 3) The rekeyed events are copartitioned with table2:
>>>
>>> a) Stream Thread with Partition 0:
>>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>>> Table2: <A,X>
>>>
>>> b) Stream Thread with Partition 1:
>>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>>> Table2: <B,Y>
>>>
>>> 4) From here, they can be joined together locally by applying the joiner
>>> function.
>>>
>>>
>>>
>>> At this point, Jan's design and my design deviate. My design goes on to
>>> repartition the data post-join and resolve out-of-order arrival of
>>> records,
>>> finally returning the data keyed just the original key. I do not expose
>>> the
>>> CombinedKey or any of the internals outside of the joinOnForeignKey
>>> function. This does make for larger footprint, but it removes all agency
>>> for resolving out-of-order arrivals and handling CombinedKeys from the
>>> user. I believe that this makes the function much easier to use.
>>>
>>> Let me know if this helps resolve your questions, and please feel free to
>>> add anything else on your mind.
>>>
>>> Thanks again,
>>> Adam
>>>
>>>
>>>
>>>
>>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>> Hi,
>>>> I am just catching up on this thread. I did not read everything so far,
>>>> but want to share couple of initial thoughts:
>>>>
>>>>
>>>>
>>>> Headers: I think there is a fundamental difference between header usage
>>>> in this KIP and KP-258. For 258, we add headers to changelog topic that
>>>> are owned by Kafka Streams and nobody else is supposed to write into
>>>> them. In fact, no user header are written into the changelog topic and
>>>> thus, there are not conflicts.
>>>>
>>>> Nevertheless, I don't see a big issue with using headers within Streams.
>>>> As long as we document it, we can have some "reserved" header keys and
>>>> users are not allowed to use when processing data with Kafka Streams.
>>>> IMHO, this should be ok.
>>>>
>>>> I think there is a safe way to avoid conflicts, since these headers are
>>>>> only needed in internal topics (I think):
>>>>> For internal and changelog topics, we can namespace all headers:
>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>
>>>> While name spacing would be possible, it would require to deserialize
>>>> user headers what implies a runtime overhead. I would suggest to no
>>>> namespace for now to avoid the overhead. If this becomes a problem in
>>>> the future, we can still add name spacing later on.
>>>>
>>>>
>>>>
>>>> My main concern about the design it the type of the result KTable: If I
>>>> understood the proposal correctly,
>>>>
>>>> KTable<K1,V1> table1 = ...
>>>> KTable<K2,V2> table2 = ...
>>>>
>>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>>
>>>> implies that the `joinedTable` has the same key as the left input table.
>>>> IMHO, this does not work because if table2 contains multiple rows that
>>>> join with a record in table1 (what is the main purpose of a foreign key
>>>> join), the result table would only contain a single join result, but not
>>>> multiple.
>>>>
>>>> Example:
>>>>
>>>> table1 input stream: <A,X>
>>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>>
>>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
>>>> result key is the same key as key of table1, this implies that the
>>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>>> Because the share the same key, whatever result record we emit later,
>>>> overwrite the previous result.
>>>>
>>>> This is the reason why Jan originally proposed to use a combination of
>>>> both primary keys of the input tables as key of the output table. This
>>>> makes the keys of the output table unique and we can store both in the
>>>> output table:
>>>>
>>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>>
>>>>
>>>> Thoughts?
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>>
>>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>>
>>>>> Just on remark here.
>>>>> The high-watermark could be disregarded. The decision about the forward
>>>>> depends on the size of the aggregated map.
>>>>> Only 1 element long maps would be unpacked and forwarded. 0 element maps
>>>>> would be published as delete. Any other count
>>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>>
>>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>>
>>>>>> It does look like I could replace the second repartition store and
>>>>>> highwater store with a groupBy and reduce.  However, it looks like I
>>>>>> would
>>>>>> still need to store the highwater value within the materialized store,
>>>>>>
>>>>> to
>>>>> compare the arrival of out-of-order records (assuming my understanding
>>>>> of
>>>>> THIS is correct...). This in effect is the same as the design I have
>>>>> now,
>>>>> just with the two tables merged together.


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
I'm currently testing using a Windowed Store to store the highwater mark.
By all indications this should work fine, with the caveat being that it can
only resolve out-of-order arrival for up to the size of the window (ie:
24h, 72h, etc). This would remove the possibility of it being unbounded in
size.

With regards to Jan's suggestion, I believe this is where we will have to
remain in disagreement. While I do not disagree with your statement about
there likely to be additional joins done in a real-world workflow, I do not
see how you can conclusively deal with out-of-order arrival of foreign-key
changes and subsequent joins. I have attempted what I think you have
proposed (without a high-water, using groupBy and reduce) and found that if
the foreign key changes too quickly, or the load on a stream thread is too
high, the joined messages will arrive out-of-order and be incorrectly
propagated, such that an intermediate event is represented as the final
event. Repartitioning the scattered events back to their original
partitions is the only way I know how to conclusively deal with
out-of-order events in a given time frame, and to ensure that the data is
eventually consistent with the input events.

If you have some code to share that illustrates your approach, I would be
very grateful as it would remove any misunderstandings that I may have.

Thanks,

Adam





On Wed, Sep 5, 2018 at 3:35 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Thanks Adam for bringing Matthias to speed!
>
> about the differences. I think re-keying back should be optional at best.
> I would say we return a KScatteredTable with reshuffle() returning
> KTable<originalKey,Joined> to make the backwards repartitioning optional.
> I am also in a big favour of doing the out of order processing using group
> by instead high water mark tracking.
> Just because unbounded growth is just scary + It saves us the header stuff.
>
> I think the abstraction of always repartitioning back is just not so
> strong. Like the work has been done before we partition back and grouping
> by something else afterwards is really common.
>
>
>
>
>
>
> On 05.09.2018 13:49, Adam Bellemare wrote:
>
>> Hi Matthias
>>
>> Thank you for your feedback, I do appreciate it!
>>
>> While name spacing would be possible, it would require to deserialize
>>> user headers what implies a runtime overhead. I would suggest to no
>>> namespace for now to avoid the overhead. If this becomes a problem in
>>> the future, we can still add name spacing later on.
>>>
>> Agreed. I will go with using a reserved string and document it.
>>
>>
>>
>> My main concern about the design it the type of the result KTable: If I
>>>
>> understood the proposal correctly,
>>
>>
>> In your example, you have table1 and table2 swapped. Here is how it works
>> currently:
>>
>> 1) table1 has the records that contain the foreign key within their value.
>> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
>> table2 input stream: <A,X>, <B,Y>
>>
>> 2) A Value mapper is required to extract the foreign key.
>> table1 foreign key mapper: ( value => value.fk )
>>
>> The mapper is applied to each element in table1, and a new combined key is
>> made:
>> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c,
>> (fk=B,bar=3)>
>>
>> 3) The rekeyed events are copartitioned with table2:
>>
>> a) Stream Thread with Partition 0:
>> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
>> Table2: <A,X>
>>
>> b) Stream Thread with Partition 1:
>> RepartitionedTable1: <B-c, (fk=B,bar=3)>
>> Table2: <B,Y>
>>
>> 4) From here, they can be joined together locally by applying the joiner
>> function.
>>
>>
>>
>> At this point, Jan's design and my design deviate. My design goes on to
>> repartition the data post-join and resolve out-of-order arrival of
>> records,
>> finally returning the data keyed just the original key. I do not expose
>> the
>> CombinedKey or any of the internals outside of the joinOnForeignKey
>> function. This does make for larger footprint, but it removes all agency
>> for resolving out-of-order arrivals and handling CombinedKeys from the
>> user. I believe that this makes the function much easier to use.
>>
>> Let me know if this helps resolve your questions, and please feel free to
>> add anything else on your mind.
>>
>> Thanks again,
>> Adam
>>
>>
>>
>>
>> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>> Hi,
>>>
>>> I am just catching up on this thread. I did not read everything so far,
>>> but want to share couple of initial thoughts:
>>>
>>>
>>>
>>> Headers: I think there is a fundamental difference between header usage
>>> in this KIP and KP-258. For 258, we add headers to changelog topic that
>>> are owned by Kafka Streams and nobody else is supposed to write into
>>> them. In fact, no user header are written into the changelog topic and
>>> thus, there are not conflicts.
>>>
>>> Nevertheless, I don't see a big issue with using headers within Streams.
>>> As long as we document it, we can have some "reserved" header keys and
>>> users are not allowed to use when processing data with Kafka Streams.
>>> IMHO, this should be ok.
>>>
>>> I think there is a safe way to avoid conflicts, since these headers are
>>>> only needed in internal topics (I think):
>>>> For internal and changelog topics, we can namespace all headers:
>>>> * user-defined headers are namespaced as "external." + headerKey
>>>> * internal headers are namespaced as "internal." + headerKey
>>>>
>>> While name spacing would be possible, it would require to deserialize
>>> user headers what implies a runtime overhead. I would suggest to no
>>> namespace for now to avoid the overhead. If this becomes a problem in
>>> the future, we can still add name spacing later on.
>>>
>>>
>>>
>>> My main concern about the design it the type of the result KTable: If I
>>> understood the proposal correctly,
>>>
>>> KTable<K1,V1> table1 = ...
>>> KTable<K2,V2> table2 = ...
>>>
>>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>>
>>> implies that the `joinedTable` has the same key as the left input table.
>>> IMHO, this does not work because if table2 contains multiple rows that
>>> join with a record in table1 (what is the main purpose of a foreign key
>>> join), the result table would only contain a single join result, but not
>>> multiple.
>>>
>>> Example:
>>>
>>> table1 input stream: <A,X>
>>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>>
>>> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
>>> result key is the same key as key of table1, this implies that the
>>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>>> Because the share the same key, whatever result record we emit later,
>>> overwrite the previous result.
>>>
>>> This is the reason why Jan originally proposed to use a combination of
>>> both primary keys of the input tables as key of the output table. This
>>> makes the keys of the output table unique and we can store both in the
>>> output table:
>>>
>>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>>
>>>
>>> Thoughts?
>>>
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>>
>>>> Just on remark here.
>>>> The high-watermark could be disregarded. The decision about the forward
>>>> depends on the size of the aggregated map.
>>>> Only 1 element long maps would be unpacked and forwarded. 0 element maps
>>>> would be published as delete. Any other count
>>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>>
>>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>>
>>>>> It does look like I could replace the second repartition store and
>>>>> highwater store with a groupBy and reduce.  However, it looks like I
>>>>> would
>>>>> still need to store the highwater value within the materialized store,
>>>>>
>>>> to
>>>
>>>> compare the arrival of out-of-order records (assuming my understanding
>>>>>
>>>> of
>>>
>>>> THIS is correct...). This in effect is the same as the design I have
>>>>>
>>>> now,
>>>
>>>> just with the two tables merged together.
>>>>>
>>>>
>>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Thanks Adam for bringing Matthias to speed!

about the differences. I think re-keying back should be optional at best.
I would say we return a KScatteredTable with reshuffle() returning 
KTable<originalKey,Joined> to make the backwards repartitioning optional.
I am also in a big favour of doing the out of order processing using 
group by instead high water mark tracking.
Just because unbounded growth is just scary + It saves us the header stuff.

I think the abstraction of always repartitioning back is just not so 
strong. Like the work has been done before we partition back and 
grouping by something else afterwards is really common.





On 05.09.2018 13:49, Adam Bellemare wrote:
> Hi Matthias
>
> Thank you for your feedback, I do appreciate it!
>
>> While name spacing would be possible, it would require to deserialize
>> user headers what implies a runtime overhead. I would suggest to no
>> namespace for now to avoid the overhead. If this becomes a problem in
>> the future, we can still add name spacing later on.
> Agreed. I will go with using a reserved string and document it.
>
>
>
>> My main concern about the design it the type of the result KTable: If I
> understood the proposal correctly,
>
>
> In your example, you have table1 and table2 swapped. Here is how it works
> currently:
>
> 1) table1 has the records that contain the foreign key within their value.
> table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
> table2 input stream: <A,X>, <B,Y>
>
> 2) A Value mapper is required to extract the foreign key.
> table1 foreign key mapper: ( value => value.fk )
>
> The mapper is applied to each element in table1, and a new combined key is
> made:
> table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c, (fk=B,bar=3)>
>
> 3) The rekeyed events are copartitioned with table2:
>
> a) Stream Thread with Partition 0:
> RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
> Table2: <A,X>
>
> b) Stream Thread with Partition 1:
> RepartitionedTable1: <B-c, (fk=B,bar=3)>
> Table2: <B,Y>
>
> 4) From here, they can be joined together locally by applying the joiner
> function.
>
>
>
> At this point, Jan's design and my design deviate. My design goes on to
> repartition the data post-join and resolve out-of-order arrival of records,
> finally returning the data keyed just the original key. I do not expose the
> CombinedKey or any of the internals outside of the joinOnForeignKey
> function. This does make for larger footprint, but it removes all agency
> for resolving out-of-order arrivals and handling CombinedKeys from the
> user. I believe that this makes the function much easier to use.
>
> Let me know if this helps resolve your questions, and please feel free to
> add anything else on your mind.
>
> Thanks again,
> Adam
>
>
>
>
> On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> Hi,
>>
>> I am just catching up on this thread. I did not read everything so far,
>> but want to share couple of initial thoughts:
>>
>>
>>
>> Headers: I think there is a fundamental difference between header usage
>> in this KIP and KP-258. For 258, we add headers to changelog topic that
>> are owned by Kafka Streams and nobody else is supposed to write into
>> them. In fact, no user header are written into the changelog topic and
>> thus, there are not conflicts.
>>
>> Nevertheless, I don't see a big issue with using headers within Streams.
>> As long as we document it, we can have some "reserved" header keys and
>> users are not allowed to use when processing data with Kafka Streams.
>> IMHO, this should be ok.
>>
>>> I think there is a safe way to avoid conflicts, since these headers are
>>> only needed in internal topics (I think):
>>> For internal and changelog topics, we can namespace all headers:
>>> * user-defined headers are namespaced as "external." + headerKey
>>> * internal headers are namespaced as "internal." + headerKey
>> While name spacing would be possible, it would require to deserialize
>> user headers what implies a runtime overhead. I would suggest to no
>> namespace for now to avoid the overhead. If this becomes a problem in
>> the future, we can still add name spacing later on.
>>
>>
>>
>> My main concern about the design it the type of the result KTable: If I
>> understood the proposal correctly,
>>
>> KTable<K1,V1> table1 = ...
>> KTable<K2,V2> table2 = ...
>>
>> KTable<K1,V3> joinedTable = table1.join(table2,...);
>>
>> implies that the `joinedTable` has the same key as the left input table.
>> IMHO, this does not work because if table2 contains multiple rows that
>> join with a record in table1 (what is the main purpose of a foreign key
>> join), the result table would only contain a single join result, but not
>> multiple.
>>
>> Example:
>>
>> table1 input stream: <A,X>
>> table2 input stream: <a,(A,1)>, <b,(A,2)>
>>
>> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
>> result key is the same key as key of table1, this implies that the
>> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
>> Because the share the same key, whatever result record we emit later,
>> overwrite the previous result.
>>
>> This is the reason why Jan originally proposed to use a combination of
>> both primary keys of the input tables as key of the output table. This
>> makes the keys of the output table unique and we can store both in the
>> output table:
>>
>> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>>
>>
>> Thoughts?
>>
>>
>> -Matthias
>>
>>
>>
>>
>> On 9/4/18 1:36 PM, Jan Filipiak wrote:
>>> Just on remark here.
>>> The high-watermark could be disregarded. The decision about the forward
>>> depends on the size of the aggregated map.
>>> Only 1 element long maps would be unpacked and forwarded. 0 element maps
>>> would be published as delete. Any other count
>>> of map entries is in "waiting for correct deletes to arrive"-state.
>>>
>>> On 04.09.2018 21:29, Adam Bellemare wrote:
>>>> It does look like I could replace the second repartition store and
>>>> highwater store with a groupBy and reduce.  However, it looks like I
>>>> would
>>>> still need to store the highwater value within the materialized store,
>> to
>>>> compare the arrival of out-of-order records (assuming my understanding
>> of
>>>> THIS is correct...). This in effect is the same as the design I have
>> now,
>>>> just with the two tables merged together.
>>


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Matthias

Thank you for your feedback, I do appreciate it!

> While name spacing would be possible, it would require to deserialize
> user headers what implies a runtime overhead. I would suggest to no
> namespace for now to avoid the overhead. If this becomes a problem in
> the future, we can still add name spacing later on.

Agreed. I will go with using a reserved string and document it.



> My main concern about the design it the type of the result KTable: If I
understood the proposal correctly,


In your example, you have table1 and table2 swapped. Here is how it works
currently:

1) table1 has the records that contain the foreign key within their value.
table1 input stream: <a,(fk=A,bar=1)>, <b,(fk=A,bar=2)>, <c,(fk=B,bar=3)>
table2 input stream: <A,X>, <B,Y>

2) A Value mapper is required to extract the foreign key.
table1 foreign key mapper: ( value => value.fk )

The mapper is applied to each element in table1, and a new combined key is
made:
table1 mapped: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>, <B-c, (fk=B,bar=3)>

3) The rekeyed events are copartitioned with table2:

a) Stream Thread with Partition 0:
RepartitionedTable1: <A-a, (fk=A,bar=1)>, <A-b, (fk=A,bar=2)>
Table2: <A,X>

b) Stream Thread with Partition 1:
RepartitionedTable1: <B-c, (fk=B,bar=3)>
Table2: <B,Y>

4) From here, they can be joined together locally by applying the joiner
function.



At this point, Jan's design and my design deviate. My design goes on to
repartition the data post-join and resolve out-of-order arrival of records,
finally returning the data keyed just the original key. I do not expose the
CombinedKey or any of the internals outside of the joinOnForeignKey
function. This does make for larger footprint, but it removes all agency
for resolving out-of-order arrivals and handling CombinedKeys from the
user. I believe that this makes the function much easier to use.

Let me know if this helps resolve your questions, and please feel free to
add anything else on your mind.

Thanks again,
Adam




On Tue, Sep 4, 2018 at 8:36 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> Hi,
>
> I am just catching up on this thread. I did not read everything so far,
> but want to share couple of initial thoughts:
>
>
>
> Headers: I think there is a fundamental difference between header usage
> in this KIP and KP-258. For 258, we add headers to changelog topic that
> are owned by Kafka Streams and nobody else is supposed to write into
> them. In fact, no user header are written into the changelog topic and
> thus, there are not conflicts.
>
> Nevertheless, I don't see a big issue with using headers within Streams.
> As long as we document it, we can have some "reserved" header keys and
> users are not allowed to use when processing data with Kafka Streams.
> IMHO, this should be ok.
>
> > I think there is a safe way to avoid conflicts, since these headers are
> > only needed in internal topics (I think):
> > For internal and changelog topics, we can namespace all headers:
> > * user-defined headers are namespaced as "external." + headerKey
> > * internal headers are namespaced as "internal." + headerKey
>
> While name spacing would be possible, it would require to deserialize
> user headers what implies a runtime overhead. I would suggest to no
> namespace for now to avoid the overhead. If this becomes a problem in
> the future, we can still add name spacing later on.
>
>
>
> My main concern about the design it the type of the result KTable: If I
> understood the proposal correctly,
>
> KTable<K1,V1> table1 = ...
> KTable<K2,V2> table2 = ...
>
> KTable<K1,V3> joinedTable = table1.join(table2,...);
>
> implies that the `joinedTable` has the same key as the left input table.
> IMHO, this does not work because if table2 contains multiple rows that
> join with a record in table1 (what is the main purpose of a foreign key
> join), the result table would only contain a single join result, but not
> multiple.
>
> Example:
>
> table1 input stream: <A,X>
> table2 input stream: <a,(A,1)>, <b,(A,2)>
>
> We use table2 value a foreign key to table1 key (ie, "A" joins). If the
> result key is the same key as key of table1, this implies that the
> result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
> Because the share the same key, whatever result record we emit later,
> overwrite the previous result.
>
> This is the reason why Jan originally proposed to use a combination of
> both primary keys of the input tables as key of the output table. This
> makes the keys of the output table unique and we can store both in the
> output table:
>
> Result would be <A-a, join(X,1)>, <A-b, join(X,2)>
>
>
> Thoughts?
>
>
> -Matthias
>
>
>
>
> On 9/4/18 1:36 PM, Jan Filipiak wrote:
> > Just on remark here.
> > The high-watermark could be disregarded. The decision about the forward
> > depends on the size of the aggregated map.
> > Only 1 element long maps would be unpacked and forwarded. 0 element maps
> > would be published as delete. Any other count
> > of map entries is in "waiting for correct deletes to arrive"-state.
> >
> > On 04.09.2018 21:29, Adam Bellemare wrote:
> >> It does look like I could replace the second repartition store and
> >> highwater store with a groupBy and reduce.  However, it looks like I
> >> would
> >> still need to store the highwater value within the materialized store,
> to
> >> compare the arrival of out-of-order records (assuming my understanding
> of
> >> THIS is correct...). This in effect is the same as the design I have
> now,
> >> just with the two tables merged together.
> >
>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Hi,

I am just catching up on this thread. I did not read everything so far,
but want to share couple of initial thoughts:



Headers: I think there is a fundamental difference between header usage
in this KIP and KP-258. For 258, we add headers to changelog topic that
are owned by Kafka Streams and nobody else is supposed to write into
them. In fact, no user header are written into the changelog topic and
thus, there are not conflicts.

Nevertheless, I don't see a big issue with using headers within Streams.
As long as we document it, we can have some "reserved" header keys and
users are not allowed to use when processing data with Kafka Streams.
IMHO, this should be ok.

> I think there is a safe way to avoid conflicts, since these headers are
> only needed in internal topics (I think):
> For internal and changelog topics, we can namespace all headers:
> * user-defined headers are namespaced as "external." + headerKey
> * internal headers are namespaced as "internal." + headerKey

While name spacing would be possible, it would require to deserialize
user headers what implies a runtime overhead. I would suggest to no
namespace for now to avoid the overhead. If this becomes a problem in
the future, we can still add name spacing later on.



My main concern about the design it the type of the result KTable: If I
understood the proposal correctly,

KTable<K1,V1> table1 = ...
KTable<K2,V2> table2 = ...

KTable<K1,V3> joinedTable = table1.join(table2,...);

implies that the `joinedTable` has the same key as the left input table.
IMHO, this does not work because if table2 contains multiple rows that
join with a record in table1 (what is the main purpose of a foreign key
join), the result table would only contain a single join result, but not
multiple.

Example:

table1 input stream: <A,X>
table2 input stream: <a,(A,1)>, <b,(A,2)>

We use table2 value a foreign key to table1 key (ie, "A" joins). If the
result key is the same key as key of table1, this implies that the
result can either be <A, join(X,1)> or <A, join(X,2)> but not both.
Because the share the same key, whatever result record we emit later,
overwrite the previous result.

This is the reason why Jan originally proposed to use a combination of
both primary keys of the input tables as key of the output table. This
makes the keys of the output table unique and we can store both in the
output table:

Result would be <A-a, join(X,1)>, <A-b, join(X,2)>


Thoughts?


-Matthias




On 9/4/18 1:36 PM, Jan Filipiak wrote:
> Just on remark here.
> The high-watermark could be disregarded. The decision about the forward
> depends on the size of the aggregated map.
> Only 1 element long maps would be unpacked and forwarded. 0 element maps
> would be published as delete. Any other count
> of map entries is in "waiting for correct deletes to arrive"-state.
> 
> On 04.09.2018 21:29, Adam Bellemare wrote:
>> It does look like I could replace the second repartition store and
>> highwater store with a groupBy and reduce.  However, it looks like I
>> would
>> still need to store the highwater value within the materialized store, to
>> compare the arrival of out-of-order records (assuming my understanding of
>> THIS is correct...). This in effect is the same as the design I have now,
>> just with the two tables merged together.
> 


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
I was completely brain-dead in my mail before.
Completly missed that you already repartition back for the user and only 
apply the high water mark filtering after the second repartition source.
I missed the sink / source bounce while poking, sorry for the confusion.

Yes a group by can give the same semantics w/o the ever growing 
highwatermark store. I made it optional
so that when users wanna group by something different after the join 
they still can and it will shuffle 1 times less.

I felt that was / is usefully and we sometimes exploit this facts in 
some of our joins, sometimes we do aggregate back to
the "original key" aswell.

I have a hard time estimating the implications of the ever growing 
highwatermark store. From the top of my head
there wouldn't be a current use case where it would be a concern for us. 
That might be different for other users though.

Sorry for the confusion!

Best Jan

PS planning to put the comments regarding java stuff on the PR.




On 04.09.2018 21:29, Adam Bellemare wrote:
> Yep, I definitely misunderstood some of the groupBy and groupByKey
> functionality. I would say disregard what I said in my previous email
> w.r.t. my assumptions about record size. I was looking into the code more
> today and I did not understand it correctly the first time I read it.
>
> It does look like I could replace the second repartition store and
> highwater store with a groupBy and reduce.  However, it looks like I would
> still need to store the highwater value within the materialized store, to
> compare the arrival of out-of-order records (assuming my understanding of
> THIS is correct...). This in effect is the same as the design I have now,
> just with the two tables merged together.
>
> I will keep looking at this but I am not seeing a great simplification.
> Advice and comments are welcomed as always.
>
> On Tue, Sep 4, 2018 at 9:38 AM, Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> As I was looking more into RocksDB TTL, I see that we currently do not
>> support it in Kafka Streams due to a number of technical reasons. As I
>> don't think that I will be tackling that JIRA at the moment, the current
>> implementation is indeed unbounded in the highwater table growth.
>>
>> An alternate option may be to replace the highwater mark table with a
>> groupBy and then perform a reduce/aggregate. My main concern here is that
>> technically we could have an unbounded amount of data to group together by
>> key, and the grouped size could exceed the Kafka maximum record size. When
>> I built the highwater mark table my intent was to work around this
>> possibility, as each record is evaluated independently and record sizing
>> issues do not come into play. If I am incorrect in this assumption, please
>> correct me, because I am a bit fuzzy on exactly how the groupBy currently
>> works.
>>
>> Any thoughts on this are appreciated. I will revisit it again when I have
>> a bit more time.
>>
>> Thanks
>>
>>
>>
>> On Mon, Sep 3, 2018 at 4:55 PM, Adam Bellemare <ad...@gmail.com>
>> wrote:
>>
>>> Hi Jan
>>>
>>> Thank you for taking the time to look into my PR. I have updated it
>>> accordingly along with the suggestions from John. Please note that I am by
>>> no means an expert on Java, so I really do appreciate any Java-specific
>>> feedback you may have. Do not worry about being overly verbose on it.
>>>
>>> You are correct with regards to the highwater mark growing unbounded. One
>>> option would be to implement the rocksDB TTL to expire records. I am open
>>> to other options as well.
>>>
>>> I have tried to detail the reasoning behind it in the KIP - I have added
>>> additional comments and I hope that it is clearer now.
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>>> Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjo
>>> ininginKTable-MultipleRapidForeign-KeyValueChanges-Whyahighw
>>> atertableisrequired.
>>>
>>> Please keep in mind that there may be something about ordering guarantees
>>> that I am not aware of. As far as I know, once you begin to operate on
>>> events in parallel across different nodes within the processor API, there
>>> are no ordering guarantees and everything is simple first-come,
>>> first-served(processed). If this is not the case then I am unaware of that
>>> fact.
>>>
>>>
>>>
>>> Thanks
>>>
>>> Adam
>>>
>>>
>>>
>>>
>>>
>>> On Mon, Sep 3, 2018 at 8:38 AM, Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> Finished my deeper scan on your approach.
>>>> Most of the comments I put at the PR are minor code style things.
>>>> One forward call seems to be a bug though, would be great if you could
>>>> double check.
>>>>
>>>> the one problem I see is that the high watermark store grows unbounded.
>>>> A key being deleted from the source table does not lead to deletion in
>>>> the watermark store.
>>>>
>>>> I also don't quite grasp the concept why it's needed.  I think the whole
>>>> offset part can go away?
>>>> It seems to deal with node failures of some kind but everything should
>>>> turn out okay without it?
>>>>
>>>> Best Jan
>>>>
>>>>
>>>> On 01.09.2018 20:44, Guozhang Wang wrote:
>>>>
>>>>> Yes Adam, that makes sense.
>>>>>
>>>>> I think it may be better to have a working PR to review before we
>>>>> complete
>>>>> the VOTE thread. In my previous experience a large feature like this are
>>>>> mostly definitely going to miss some devils in the details in the design
>>>>> and wiki discussion phases.
>>>>>
>>>>> That would unfortunately mean that your implementations may need to be
>>>>> modified / updated along with the review and further KIP discussion. I
>>>>> can
>>>>> understand this can be painful, but that may be the best option we can
>>>>> do
>>>>> to avoid as much work to be wasted as possible.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <
>>>>> adam.bellemare@gmail.com>
>>>>> wrote:
>>>>>
>>>>> Hi Guozhang
>>>>>> By workflow I mean just the overall process of how the KIP is
>>>>>> implemented.
>>>>>> Any ideas on the ways to reduce the topic count, materializations, if
>>>>>> there
>>>>>> is a better way to resolve out-of-order than a highwater mark table,
>>>>>> if the
>>>>>> design philosophy of “keep everything encapsulated within the join
>>>>>> function” is appropriate, etc. I can implement the changes that John
>>>>>> suggested, but if my overall workflow is not acceptable I would rather
>>>>>> address that before making minor changes.
>>>>>>
>>>>>> If this requires a full candidate PR ready to go to prod then I can
>>>>>> make
>>>>>> those changes. Hope that clears things up.
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>> On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Adam,
>>>>>>>
>>>>>>> What do you mean by "additional comments on the workflow.", do you
>>>>>>> mean
>>>>>>>
>>>>>> to
>>>>>>
>>>>>>> let other review your PR https://github.com/apache/kafka/pull/5527 ?
>>>>>>> Is
>>>>>>>
>>>>>> is
>>>>>>
>>>>>>> ready for reviews?
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>> On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
>>>>>>>
>>>>>> adam.bellemare@gmail.com>
>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Okay, I will implement John's suggestion of namespacing the external
>>>>>>>> headers prior to processing, and then removing the namespacing prior
>>>>>>>> to
>>>>>>>> emitting. A potential future KIP could be to provide this namespacing
>>>>>>>> automatically.
>>>>>>>>
>>>>>>>> I would also appreciate any other additional comments on the
>>>>>>>> workflow.
>>>>>>>>
>>>>>>> My
>>>>>>> goal is suss out agreement prior to moving to a vote.
>>>>>>>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>>> wrote:
>>>>>>> I like John's idea as well: for this KIP specifically as we do not
>>>>>>>> expect
>>>>>>> any other consumers to read the repartition topics externally, we can
>>>>>>>>> slightly prefix the header to be safe, while keeping the additional
>>>>>>>>>
>>>>>>>> cost
>>>>>>> (note the header field is per-record, so any additional byte is
>>>>>>>> per-record
>>>>>>>>
>>>>>>>>> as well) low.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
>>>>>>>>>
>>>>>>>> adam.bellemare@gmail.com
>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Hi John
>>>>>>>>>> That is an excellent idea. The header usage I propose would be
>>>>>>>>>> limited
>>>>>>>>>> entirely to internal topics, and this could very well be the
>>>>>>>>>> solution
>>>>>>>>>>
>>>>>>>>> to
>>>>>>>>> potential conflicts. If we do not officially reserve a prefix "__"
>>>>>>>>> then I
>>>>>>>>> think this would be the safest idea, as it would entirely avoid any
>>>>>>>>>> accidents (perhaps if a company is using its own "__" prefix for
>>>>>>>>>> other
>>>>>>>>>> reasons).
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>>
>>>>>>>>>> Adam
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
>>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>> Just a quick thought regarding headers:
>>>>>>>>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
>>>>>>>>>>>>
>>>>>>>>>>> can
>>>>>>>>> still
>>>>>>>>>>>> consider using some name patterns to reduce the likelihood as
>>>>>>>>>>>> much
>>>>>>>>>>>>
>>>>>>>>>>> as
>>>>>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
>>>>>>>>>>>> "__internal_[name]"?
>>>>>>>>>>>>
>>>>>>>>>>> I think there is a safe way to avoid conflicts, since these
>>>>>>>>>>> headers
>>>>>>>>>>>
>>>>>>>>>> are
>>>>>>>>> only needed in internal topics (I think):
>>>>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>>>>
>>>>>>>>>>> This is a lot of characters, so we could use a sigil instead
>>>>>>>>>>> (e.g.,
>>>>>>>>>>>
>>>>>>>>>> "_"
>>>>>>>>> for
>>>>>>>>>>> internal, "~" for external)
>>>>>>>>>>>
>>>>>>>>>>> We simply apply the namespacing when we read user headers from
>>>>>>>>>>>
>>>>>>>>>> external
>>>>>>>>> topics into the topology and then de-namespace them before we emit
>>>>>>>>>> them
>>>>>>>>> to
>>>>>>>>>>> an external topic (via "to" or "through").
>>>>>>>>>>> Now, it is not possible to collide with user-defined headers.
>>>>>>>>>>>
>>>>>>>>>>> That said, I'd also be fine with just reserving "__" as a header
>>>>>>>>>>>
>>>>>>>>>> prefix
>>>>>>>>> and
>>>>>>>>>>> not worrying about collisions.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for the KIP,
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
>>>>>>>>>>>
>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>> wrote:
>>>>>>>>>>> Still havent completly grabbed it.
>>>>>>>>>>>> sorry will read more
>>>>>>>>>>>>
>>>>>>>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
>>>>>>>>>>>>> Cool stuff.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I made some random remarks. Did not touch the core of the
>>>>>>>>>>>>>
>>>>>>>>>>>> algorithm
>>>>>>>>> yet.
>>>>>>>>>>>> Will do Monday 100%
>>>>>>>>>>>>> I don't see Interactive Queries :) like that!
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
>>>>>>>>>>>>>> I have submitted a PR with my code against trunk:
>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Do I continue on this thread or do we begin a new one for
>>>>>>>>>>>>>>
>>>>>>>>>>>>> discussion?
>>>>>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> even before message headers, the option for me always existed
>>>>>>>>>>>>>> to
>>>>>>>>> just wrap
>>>>>>>>>>>>>>> the messages into my own custom envelop.
>>>>>>>>>>>>>>> So I of course thought this through. One sentence in your last
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> email
>>>>>>>>>>> triggered all the thought process I put in the back then
>>>>>>>>>>>>>>> again to design it in the, what i think is the "kafka-way". It
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ended
>>>>>>>>>>> up
>>>>>>>>>>>
>>>>>>>>>>>> ranting a little about what happened in the past.
>>>>>>>>>>>>>>> I see plenty of colleagues of mine falling into traps in the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> API,
>>>>>>>>> that I
>>>>>>>>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
>>>>>>>>>>>>>>> feeling again. So I hope it gives you some insights into my
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> though
>>>>>>>>>> process. I am aware that since i never ported 213 to higher
>>>>>>>>>>>>>>> streams version, I don't really have a steak here and
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> initially I
>>>>>>>>> didn't
>>>>>>>>>>>>>>> feel like actually sending it. But maybe you can pull
>>>>>>>>>>>>>>> something good from it.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>    Best jan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
>>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow
>>>>>>>>>>>>>>>> includes
>>>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>> about which record is processed first? I understand about a
>>>>>>>>>>>>>>>> CombinedKey
>>>>>>>>>>>>>>>> with both the Foreign and Primary key, but I don't see how
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> track
>>>>>>>>>>> ordering metadata in there unless you actually included a
>>>>>>>>>>>>>>> metadata
>>>>>>>>>> field
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> the key type as well.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> be
>>>>>>>>> used in
>>>>>>>>>>>>>>>> just the user-space? It seems that it is possible that a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> collision
>>>>>>>>>> on the
>>>>>>>>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> instance,
>>>>>>>>> if
>>>>>>>>>>> I
>>>>>>>>>>>
>>>>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
>>>>>>>>>>>>>>> user
>>>>>>>>>>> already
>>>>>>>>>>>>>>>> specified their own header with the same key name, then it
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> appears
>>>>>>>>>> there
>>>>>>>>>>>>>>>> would be a collision. (This is one of the issues I brought up
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> in
>>>>>>>>> the KIP).
>>>>>>>>>>>>>>>> --------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I will be posting a prototype PR against trunk within the
>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> day
>>>>>>>>>> or two.
>>>>>>>>>>>>>>>> One thing I need to point out is that my design very strictly
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wraps
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> function.
>>>>>>>>> There is
>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> with
>>>>>>>>>> regards
>>>>>>>>>>>>>>>> to out-of-order processing and no need for the DSL user to
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> even
>>>>>>>>> know
>>>>>>>>>>> what's
>>>>>>>>>>>>>>>> going on inside of the function. The code simply returns the
>>>>>>>>>>>>>>>> results of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> join, keyed by the original key. Currently my API mirrors
>>>>>>>>>>>>>>>> identically the
>>>>>>>>>>>>>>>> format of the data returned by the regular join function, and
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I
>>>>>>>>> believe
>>>>>>>>>>>>>>>> that this is very useful to many users of the DSL. It is my
>>>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>> that one of the main design goals of the DSL is to provide
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> higher
>>>>>>>>>> level
>>>>>>>>>>>>>>>> functionality without requiring the users to know exactly
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> what's
>>>>>>>>> going on
>>>>>>>>>>>>>>>> under the hood. With this in mind, I thought it best to solve
>>>>>>>>>>>>>>>> ordering and
>>>>>>>>>>>>>>>> partitioning problems within the function and eliminate the
>>>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> users to do additional work after the fact to resolve the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> results
>>>>>>>>>> of their
>>>>>>>>>>>>>>>> join. Basically, I am assuming that most users of the DSL
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>> "want it to
>>>>>>>>>>>>>>>> work" and want it to be easy. I did this operating under the
>>>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>>> that if a user truly wants to optimize their own workflow
>>>>>>>>>>>>>>>> down
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>> finest details then they will break from strictly using the
>>>>>>>>>>>>>>> DSL
>>>>>>>>> and
>>>>>>>>>>> move
>>>>>>>>>>>>>>>> down to the processors API.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think. The abstraction is not powerful enough
>>>>>>>>>>>>>>> to not have kafka specifics leak up The leak I currently think
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> this
>>>>>>>>>> has is
>>>>>>>>>>>>>>> that you can not reliable prevent the delete coming out first,
>>>>>>>>>>>>>>> before you emit the correct new record. As it is an
>>>>>>>>>>>>>>> abstraction
>>>>>>>>>>>>>>> entirely
>>>>>>>>>>>>>>> around kafka.
>>>>>>>>>>>>>>> I can only recommend to not to. Honesty and simplicity should
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> always
>>>>>>>>>>> be
>>>>>>>>>>>
>>>>>>>>>>>> first prio
>>>>>>>>>>>>>>> trying to hide this just makes it more complex, less
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> understandable
>>>>>>>>>> and
>>>>>>>>>>>> will lead to mistakes
>>>>>>>>>>>>>>> in usage.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
>>>>>>>>>>>>>>> optimization stages.
>>>>>>>>>>>>>>> Can someone give me an example of an optimisation that really
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> can't
>>>>>>>>>> be
>>>>>>>>>>>> handled by the user
>>>>>>>>>>>>>>> constructing his topology differently?
>>>>>>>>>>>>>>> Having reusable Processor API components accessible by the DSL
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> and
>>>>>>>>>> composable as
>>>>>>>>>>>>>>> one likes is exactly where DSL should max out and KSQL should
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> do
>>>>>>>>> the
>>>>>>>>>>> next
>>>>>>>>>>>>>>> step.
>>>>>>>>>>>>>>> I find it very unprofessional from a software engineering
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> approach
>>>>>>>>>> to run
>>>>>>>>>>>>>>> software where
>>>>>>>>>>>>>>> you can not at least senseful reason about the inner workings
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> of
>>>>>>>>> the
>>>>>>>>>>> libraries used.
>>>>>>>>>>>>>>> Gives this people have to read and understand in anyway, why
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> try
>>>>>>>>> to
>>>>>>>>>
>>>>>>>>>> hide
>>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It really miss the beauty of 0.10 version DSL.
>>>>>>>>>>>>>>> Apparently not a thing I can influence but just warn about.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @gouzhang
>>>>>>>>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> prune
>>>>>>>>>> from
>>>>>>>>>>>> stream app's
>>>>>>>>>>>>>>> because people just keep passing Materialized's into all the
>>>>>>>>>>>>>>> operations.
>>>>>>>>>>>>>>> :D :'-(
>>>>>>>>>>>>>>> I regret that I couldn't convince you guys back then. Plus
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> whole
>>>>>>>>>>> entire topology as a floating
>>>>>>>>>>>>>>> interface chain, never seen it anywhere :-/ :'(
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't know. I guess this is just me regretting to only have
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 24h/day.
>>>>>>>>>>>>>>> I updated the KIP today with some points worth talking about,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> should
>>>>>>>>>>> anyone
>>>>>>>>>>>>>>>> be so inclined to check it out. Currently we are running this
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> code
>>>>>>>>>> in
>>>>>>>>>>>> production to handle relational joins from our Kafka Connect
>>>>>>>>>>>>>>>> topics, as
>>>>>>>>>>>>>>>> per
>>>>>>>>>>>>>>>> the original motivation of the KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As for your question regarding GraphNodes, it is for
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> extending
>>>>>>>>> Streams
>>>>>>>>>>>>>>>>> optimization framework. You can find more details on
>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The main idea is that instead of directly building up the
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> "physical
>>>>>>>>>>> topology" (represented as Topology in the public package, and
>>>>>>>>>>>>>>>>> internally
>>>>>>>>>>>>>>>>> built as the ProcessorTopology class) while users are
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> specifying
>>>>>>>>>> the
>>>>>>>>>>>> transformation operators, we first keep it as a "logical
>>>>>>>>>>>>>>>> topology"
>>>>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
>>>>>>>>>>>>>>>> then
>>>>>>>>>>> only
>>>>>>>>>>>>>>>>> execute the optimization and the construction of the
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> "physical"
>>>>>>>>> Topology
>>>>>>>>>>>>>>>>> when StreamsBuilder.build() is called.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Back to your question, I think it makes more sense to add a
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> new
>>>>>>>>> type of
>>>>>>>>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
>>>>>>>>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> will
>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> multiple connected ProcessorNodes to represent a
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> (foreign-key)
>>>>>>>>> join, we
>>>>>>>>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> couple of
>>>>>>>>>>>>>>>>> them in the logical representation so that in the future we
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> can
>>>>>>>>> construct
>>>>>>>>>>>>>>>>> the physical topology differently (e.g. having another way
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> than
>>>>>>>>> the
>>>>>>>>>>> current
>>>>>>>>>>>>>>>>> distributed hash-join).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -------------------------------------------------------
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Back to your questions to KIP-213, I think Jan has
>>>>>>>>>>>>>>>>> summarized
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> it
>>>>>>>>>> pretty-well. Note that back then we do not have headers
>>>>>>>>>>>>>>>> support
>>>>>>>>> so
>>>>>>>>>>> we
>>>>>>>>>>>
>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
>>>>>>>>>>>>>>>>> Filipiak<Ja...@trivago.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I love how you are on to this already! I resolve this by
>>>>>>>>>>>>>>>>>> "key-widening"
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> treat the result of FKA,and FKB differently.
>>>>>>>>>>>>>>>>>> As you can see the output of my join has a Combined Key and
>>>>>>>>>>>>>>>>>> therefore I
>>>>>>>>>>>>>>>>>> can resolve the "race condition" in a group by
>>>>>>>>>>>>>>>>>> if I so desire.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think this reflects more what happens under the hood and
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> makes
>>>>>>>>>> it more
>>>>>>>>>>>>>>>>>> clear to the user what is going on. The Idea
>>>>>>>>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> from
>>>>>>>>>> my POV
>>>>>>>>>>>>>>>>>> unideal.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> To write into your example:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> key + A, null)
>>>>>>>>>>>>>>>>>> (key +B, <joined On FK =B>)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> is what my output would look like.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hope that makes sense :D
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> If you do not use headers or other metadata, how do you
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>> that
>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
>>>>>>>>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> you
>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> (FK=B).
>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>> the
>>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>>> output to be in either order - the correctly updated
>>>>>>>>>>>>>>>>>>> joined
>>>>>>>>>>>>>>>>>>> value, or
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> null for the delete.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I looked back through your code and through the discussion
>>>>>>>>>>>>>>>>>>> threads, and
>>>>>>>>>>>>>>>>>>> didn't see any information on how you resolved this. I
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> have a
>>>>>>>>> version
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>> code working for 2.0, I am just adding more integration
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> tests
>>>>>>>>> and will
>>>>>>>>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> on
>>>>>>>>> resolving
>>>>>>>>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
>>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Happy to see that you want to make an effort here.
>>>>>>>>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>> rewrite
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> joiners + the merger.
>>>>>>>>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> if
>>>>>>>>> repartition
>>>>>>>>>>>>>>>>>> is optimized in 2.0 now.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I made this
>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 241+
>>>>>>>>> KTable+repartition+with+compacted+Topics
>>>>>>>>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> combination.
>>>>>>>>>>>> For us it is vital as it minimized the size we had in our
>>>>>>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> message.
>>>>>>>>>>>> I know about this new  "delete once consumer has read it".
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
>>>>>>>>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>> am
>>>>>>>>>>> certain
>>>>>>>>>>>>>>>>>>>> that you do not need RecordHeaders
>>>>>>>>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> belong
>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>> or to the user, having a mixed use is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>> needs more.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> couldn't
>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>> can
>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>> help me see the good sides a little bit more.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If there is anything unclear with all the text that has
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> been
>>>>>>>>> written,
>>>>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
>>>>>>>>>>>>>>>>>>>> the mailing list.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me
>>>>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>>> previously.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
>>>>>>>>>>>>>>>>>>>> major
>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
>>>>>>>>>>>>>>>>> about
>>>>>>>>> this:
>>>>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
>>>>>>>>>>>>>>>>>>>> node
>>>>>>>>>>> type?
>>>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>> would it be advisable to construct it from existing
>>>>>>>>>>>>>>>>> GraphNode
>>>>>>>>> components?
>>>>>>>>>>>>>>>>>>>>



Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Just on remark here.
The high-watermark could be disregarded. The decision about the forward 
depends on the size of the aggregated map.
Only 1 element long maps would be unpacked and forwarded. 0 element maps 
would be published as delete. Any other count
of map entries is in "waiting for correct deletes to arrive"-state.

On 04.09.2018 21:29, Adam Bellemare wrote:
> It does look like I could replace the second repartition store and
> highwater store with a groupBy and reduce.  However, it looks like I would
> still need to store the highwater value within the materialized store, to
> compare the arrival of out-of-order records (assuming my understanding of
> THIS is correct...). This in effect is the same as the design I have now,
> just with the two tables merged together.


Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Yep, I definitely misunderstood some of the groupBy and groupByKey
functionality. I would say disregard what I said in my previous email
w.r.t. my assumptions about record size. I was looking into the code more
today and I did not understand it correctly the first time I read it.

It does look like I could replace the second repartition store and
highwater store with a groupBy and reduce.  However, it looks like I would
still need to store the highwater value within the materialized store, to
compare the arrival of out-of-order records (assuming my understanding of
THIS is correct...). This in effect is the same as the design I have now,
just with the two tables merged together.

I will keep looking at this but I am not seeing a great simplification.
Advice and comments are welcomed as always.

On Tue, Sep 4, 2018 at 9:38 AM, Adam Bellemare <ad...@gmail.com>
wrote:

>
> As I was looking more into RocksDB TTL, I see that we currently do not
> support it in Kafka Streams due to a number of technical reasons. As I
> don't think that I will be tackling that JIRA at the moment, the current
> implementation is indeed unbounded in the highwater table growth.
>
> An alternate option may be to replace the highwater mark table with a
> groupBy and then perform a reduce/aggregate. My main concern here is that
> technically we could have an unbounded amount of data to group together by
> key, and the grouped size could exceed the Kafka maximum record size. When
> I built the highwater mark table my intent was to work around this
> possibility, as each record is evaluated independently and record sizing
> issues do not come into play. If I am incorrect in this assumption, please
> correct me, because I am a bit fuzzy on exactly how the groupBy currently
> works.
>
> Any thoughts on this are appreciated. I will revisit it again when I have
> a bit more time.
>
> Thanks
>
>
>
> On Mon, Sep 3, 2018 at 4:55 PM, Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi Jan
>>
>> Thank you for taking the time to look into my PR. I have updated it
>> accordingly along with the suggestions from John. Please note that I am by
>> no means an expert on Java, so I really do appreciate any Java-specific
>> feedback you may have. Do not worry about being overly verbose on it.
>>
>> You are correct with regards to the highwater mark growing unbounded. One
>> option would be to implement the rocksDB TTL to expire records. I am open
>> to other options as well.
>>
>> I have tried to detail the reasoning behind it in the KIP - I have added
>> additional comments and I hope that it is clearer now.
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
>> Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjo
>> ininginKTable-MultipleRapidForeign-KeyValueChanges-Whyahighw
>> atertableisrequired.
>>
>> Please keep in mind that there may be something about ordering guarantees
>> that I am not aware of. As far as I know, once you begin to operate on
>> events in parallel across different nodes within the processor API, there
>> are no ordering guarantees and everything is simple first-come,
>> first-served(processed). If this is not the case then I am unaware of that
>> fact.
>>
>>
>>
>> Thanks
>>
>> Adam
>>
>>
>>
>>
>>
>> On Mon, Sep 3, 2018 at 8:38 AM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>>> Finished my deeper scan on your approach.
>>> Most of the comments I put at the PR are minor code style things.
>>> One forward call seems to be a bug though, would be great if you could
>>> double check.
>>>
>>> the one problem I see is that the high watermark store grows unbounded.
>>> A key being deleted from the source table does not lead to deletion in
>>> the watermark store.
>>>
>>> I also don't quite grasp the concept why it's needed.  I think the whole
>>> offset part can go away?
>>> It seems to deal with node failures of some kind but everything should
>>> turn out okay without it?
>>>
>>> Best Jan
>>>
>>>
>>> On 01.09.2018 20:44, Guozhang Wang wrote:
>>>
>>>> Yes Adam, that makes sense.
>>>>
>>>> I think it may be better to have a working PR to review before we
>>>> complete
>>>> the VOTE thread. In my previous experience a large feature like this are
>>>> mostly definitely going to miss some devils in the details in the design
>>>> and wiki discussion phases.
>>>>
>>>> That would unfortunately mean that your implementations may need to be
>>>> modified / updated along with the review and further KIP discussion. I
>>>> can
>>>> understand this can be painful, but that may be the best option we can
>>>> do
>>>> to avoid as much work to be wasted as possible.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <
>>>> adam.bellemare@gmail.com>
>>>> wrote:
>>>>
>>>> Hi Guozhang
>>>>>
>>>>> By workflow I mean just the overall process of how the KIP is
>>>>> implemented.
>>>>> Any ideas on the ways to reduce the topic count, materializations, if
>>>>> there
>>>>> is a better way to resolve out-of-order than a highwater mark table,
>>>>> if the
>>>>> design philosophy of “keep everything encapsulated within the join
>>>>> function” is appropriate, etc. I can implement the changes that John
>>>>> suggested, but if my overall workflow is not acceptable I would rather
>>>>> address that before making minor changes.
>>>>>
>>>>> If this requires a full candidate PR ready to go to prod then I can
>>>>> make
>>>>> those changes. Hope that clears things up.
>>>>>
>>>>> Thanks
>>>>>
>>>>> Adam
>>>>>
>>>>> On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>> Hi Adam,
>>>>>>
>>>>>> What do you mean by "additional comments on the workflow.", do you
>>>>>> mean
>>>>>>
>>>>> to
>>>>>
>>>>>> let other review your PR https://github.com/apache/kafka/pull/5527 ?
>>>>>> Is
>>>>>>
>>>>> is
>>>>>
>>>>>> ready for reviews?
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>> On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
>>>>>>
>>>>> adam.bellemare@gmail.com>
>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Okay, I will implement John's suggestion of namespacing the external
>>>>>>> headers prior to processing, and then removing the namespacing prior
>>>>>>> to
>>>>>>> emitting. A potential future KIP could be to provide this namespacing
>>>>>>> automatically.
>>>>>>>
>>>>>>> I would also appreciate any other additional comments on the
>>>>>>> workflow.
>>>>>>>
>>>>>> My
>>>>>
>>>>>> goal is suss out agreement prior to moving to a vote.
>>>>>>>
>>>>>>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>>>
>>>>>>> wrote:
>>>>>
>>>>>> I like John's idea as well: for this KIP specifically as we do not
>>>>>>>>
>>>>>>> expect
>>>>>
>>>>>> any other consumers to read the repartition topics externally, we can
>>>>>>>> slightly prefix the header to be safe, while keeping the additional
>>>>>>>>
>>>>>>> cost
>>>>>
>>>>>> (note the header field is per-record, so any additional byte is
>>>>>>>>
>>>>>>> per-record
>>>>>>>
>>>>>>>> as well) low.
>>>>>>>>
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
>>>>>>>>
>>>>>>> adam.bellemare@gmail.com
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Hi John
>>>>>>>>>
>>>>>>>>> That is an excellent idea. The header usage I propose would be
>>>>>>>>> limited
>>>>>>>>> entirely to internal topics, and this could very well be the
>>>>>>>>> solution
>>>>>>>>>
>>>>>>>> to
>>>>>>>
>>>>>>>> potential conflicts. If we do not officially reserve a prefix "__"
>>>>>>>>>
>>>>>>>> then I
>>>>>>>
>>>>>>>> think this would be the safest idea, as it would entirely avoid any
>>>>>>>>> accidents (perhaps if a company is using its own "__" prefix for
>>>>>>>>> other
>>>>>>>>> reasons).
>>>>>>>>>
>>>>>>>>> Thanks
>>>>>>>>>
>>>>>>>>> Adam
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
>>>>>>>>>
>>>>>>>> wrote:
>>>>>>>
>>>>>>>> Just a quick thought regarding headers:
>>>>>>>>>>
>>>>>>>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
>>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>
>>>>>>>> still
>>>>>>>>>>
>>>>>>>>>>> consider using some name patterns to reduce the likelihood as
>>>>>>>>>>> much
>>>>>>>>>>>
>>>>>>>>>> as
>>>>>>>
>>>>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
>>>>>>>>>>> "__internal_[name]"?
>>>>>>>>>>>
>>>>>>>>>> I think there is a safe way to avoid conflicts, since these
>>>>>>>>>> headers
>>>>>>>>>>
>>>>>>>>> are
>>>>>>>
>>>>>>>> only needed in internal topics (I think):
>>>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>>>
>>>>>>>>>> This is a lot of characters, so we could use a sigil instead
>>>>>>>>>> (e.g.,
>>>>>>>>>>
>>>>>>>>> "_"
>>>>>>>
>>>>>>>> for
>>>>>>>>>
>>>>>>>>>> internal, "~" for external)
>>>>>>>>>>
>>>>>>>>>> We simply apply the namespacing when we read user headers from
>>>>>>>>>>
>>>>>>>>> external
>>>>>>>
>>>>>>>> topics into the topology and then de-namespace them before we emit
>>>>>>>>>>
>>>>>>>>> them
>>>>>>>
>>>>>>>> to
>>>>>>>>>
>>>>>>>>>> an external topic (via "to" or "through").
>>>>>>>>>> Now, it is not possible to collide with user-defined headers.
>>>>>>>>>>
>>>>>>>>>> That said, I'd also be fine with just reserving "__" as a header
>>>>>>>>>>
>>>>>>>>> prefix
>>>>>>>
>>>>>>>> and
>>>>>>>>>
>>>>>>>>>> not worrying about collisions.
>>>>>>>>>>
>>>>>>>>>> Thanks for the KIP,
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
>>>>>>>>>>
>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Still havent completly grabbed it.
>>>>>>>>>>> sorry will read more
>>>>>>>>>>>
>>>>>>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
>>>>>>>>>>>> Cool stuff.
>>>>>>>>>>>>
>>>>>>>>>>>> I made some random remarks. Did not touch the core of the
>>>>>>>>>>>>
>>>>>>>>>>> algorithm
>>>>>>>
>>>>>>>> yet.
>>>>>>>>>>
>>>>>>>>>>> Will do Monday 100%
>>>>>>>>>>>>
>>>>>>>>>>>> I don't see Interactive Queries :) like that!
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
>>>>>>>>>>>>> I have submitted a PR with my code against trunk:
>>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>>
>>>>>>>>>>>>> Do I continue on this thread or do we begin a new one for
>>>>>>>>>>>>>
>>>>>>>>>>>> discussion?
>>>>>>>>>
>>>>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
>>>>>>>>>>>>>
>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> even before message headers, the option for me always existed
>>>>>>>>>>>>>>
>>>>>>>>>>>>> to
>>>>>>>
>>>>>>>> just wrap
>>>>>>>>>>>>>> the messages into my own custom envelop.
>>>>>>>>>>>>>> So I of course thought this through. One sentence in your last
>>>>>>>>>>>>>>
>>>>>>>>>>>>> email
>>>>>>>>>
>>>>>>>>>> triggered all the thought process I put in the back then
>>>>>>>>>>>>>> again to design it in the, what i think is the "kafka-way". It
>>>>>>>>>>>>>>
>>>>>>>>>>>>> ended
>>>>>>>>>
>>>>>>>>>> up
>>>>>>>>>>
>>>>>>>>>>> ranting a little about what happened in the past.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I see plenty of colleagues of mine falling into traps in the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> API,
>>>>>>>
>>>>>>>> that I
>>>>>>>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
>>>>>>>>>>>>>> feeling again. So I hope it gives you some insights into my
>>>>>>>>>>>>>>
>>>>>>>>>>>>> though
>>>>>>>>
>>>>>>>>> process. I am aware that since i never ported 213 to higher
>>>>>>>>>>>>>> streams version, I don't really have a steak here and
>>>>>>>>>>>>>>
>>>>>>>>>>>>> initially I
>>>>>>>
>>>>>>>> didn't
>>>>>>>>>>>>>> feel like actually sending it. But maybe you can pull
>>>>>>>>>>>>>> something good from it.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   Best jan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow
>>>>>>>>>>>>>>> includes
>>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>> about which record is processed first? I understand about a
>>>>>>>>>>>>>>> CombinedKey
>>>>>>>>>>>>>>> with both the Foreign and Primary key, but I don't see how
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> track
>>>>>>>>>
>>>>>>>>>> ordering metadata in there unless you actually included a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> metadata
>>>>>>>>
>>>>>>>>> field
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the key type as well.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> be
>>>>>>>
>>>>>>>> used in
>>>>>>>>>>>>>>> just the user-space? It seems that it is possible that a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> collision
>>>>>>>>
>>>>>>>>> on the
>>>>>>>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> instance,
>>>>>>>
>>>>>>>> if
>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> user
>>>>>>>>>
>>>>>>>>>> already
>>>>>>>>>>>>>>> specified their own header with the same key name, then it
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> appears
>>>>>>>>
>>>>>>>>> there
>>>>>>>>>>>>>>> would be a collision. (This is one of the issues I brought up
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> in
>>>>>>>
>>>>>>>> the KIP).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --------------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I will be posting a prototype PR against trunk within the
>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> day
>>>>>>>>
>>>>>>>>> or two.
>>>>>>>>>>>>>>> One thing I need to point out is that my design very strictly
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wraps
>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> function.
>>>>>>>
>>>>>>>> There is
>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> with
>>>>>>>>
>>>>>>>>> regards
>>>>>>>>>>>>>>> to out-of-order processing and no need for the DSL user to
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> even
>>>>>>>
>>>>>>>> know
>>>>>>>>>
>>>>>>>>>> what's
>>>>>>>>>>>>>>> going on inside of the function. The code simply returns the
>>>>>>>>>>>>>>> results of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> join, keyed by the original key. Currently my API mirrors
>>>>>>>>>>>>>>> identically the
>>>>>>>>>>>>>>> format of the data returned by the regular join function, and
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I
>>>>>>>
>>>>>>>> believe
>>>>>>>>>>>>>>> that this is very useful to many users of the DSL. It is my
>>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>> that one of the main design goals of the DSL is to provide
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> higher
>>>>>>>>
>>>>>>>>> level
>>>>>>>>>>>>>>> functionality without requiring the users to know exactly
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> what's
>>>>>>>
>>>>>>>> going on
>>>>>>>>>>>>>>> under the hood. With this in mind, I thought it best to solve
>>>>>>>>>>>>>>> ordering and
>>>>>>>>>>>>>>> partitioning problems within the function and eliminate the
>>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>> users to do additional work after the fact to resolve the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> results
>>>>>>>>
>>>>>>>>> of their
>>>>>>>>>>>>>>> join. Basically, I am assuming that most users of the DSL
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>> "want it to
>>>>>>>>>>>>>>> work" and want it to be easy. I did this operating under the
>>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>> that if a user truly wants to optimize their own workflow
>>>>>>>>>>>>>>> down
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> to
>>>>>>>>
>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>>> finest details then they will break from strictly using the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> DSL
>>>>>>>
>>>>>>>> and
>>>>>>>>>
>>>>>>>>>> move
>>>>>>>>>>>>>>> down to the processors API.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think. The abstraction is not powerful enough
>>>>>>>>>>>>>> to not have kafka specifics leak up The leak I currently think
>>>>>>>>>>>>>>
>>>>>>>>>>>>> this
>>>>>>>>
>>>>>>>>> has is
>>>>>>>>>>>>>> that you can not reliable prevent the delete coming out first,
>>>>>>>>>>>>>> before you emit the correct new record. As it is an
>>>>>>>>>>>>>> abstraction
>>>>>>>>>>>>>> entirely
>>>>>>>>>>>>>> around kafka.
>>>>>>>>>>>>>> I can only recommend to not to. Honesty and simplicity should
>>>>>>>>>>>>>>
>>>>>>>>>>>>> always
>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>>> first prio
>>>>>>>>>>>>>> trying to hide this just makes it more complex, less
>>>>>>>>>>>>>>
>>>>>>>>>>>>> understandable
>>>>>>>>
>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>>> will lead to mistakes
>>>>>>>>>>>>>> in usage.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
>>>>>>>>>>>>>> optimization stages.
>>>>>>>>>>>>>> Can someone give me an example of an optimisation that really
>>>>>>>>>>>>>>
>>>>>>>>>>>>> can't
>>>>>>>>
>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>>> handled by the user
>>>>>>>>>>>>>> constructing his topology differently?
>>>>>>>>>>>>>> Having reusable Processor API components accessible by the DSL
>>>>>>>>>>>>>>
>>>>>>>>>>>>> and
>>>>>>>>
>>>>>>>>> composable as
>>>>>>>>>>>>>> one likes is exactly where DSL should max out and KSQL should
>>>>>>>>>>>>>>
>>>>>>>>>>>>> do
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> next
>>>>>>>>>>>>>> step.
>>>>>>>>>>>>>> I find it very unprofessional from a software engineering
>>>>>>>>>>>>>>
>>>>>>>>>>>>> approach
>>>>>>>>
>>>>>>>>> to run
>>>>>>>>>>>>>> software where
>>>>>>>>>>>>>> you can not at least senseful reason about the inner workings
>>>>>>>>>>>>>>
>>>>>>>>>>>>> of
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> libraries used.
>>>>>>>>>>>>>> Gives this people have to read and understand in anyway, why
>>>>>>>>>>>>>>
>>>>>>>>>>>>> try
>>>>>>>
>>>>>>>> to
>>>>>>>>
>>>>>>>>> hide
>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It really miss the beauty of 0.10 version DSL.
>>>>>>>>>>>>>> Apparently not a thing I can influence but just warn about.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> @gouzhang
>>>>>>>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
>>>>>>>>>>>>>>
>>>>>>>>>>>>> prune
>>>>>>>>
>>>>>>>>> from
>>>>>>>>>>
>>>>>>>>>>> stream app's
>>>>>>>>>>>>>> because people just keep passing Materialized's into all the
>>>>>>>>>>>>>> operations.
>>>>>>>>>>>>>> :D :'-(
>>>>>>>>>>>>>> I regret that I couldn't convince you guys back then. Plus
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>
>>>>>>>>>>>>> whole
>>>>>>>>>
>>>>>>>>>> entire topology as a floating
>>>>>>>>>>>>>> interface chain, never seen it anywhere :-/ :'(
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't know. I guess this is just me regretting to only have
>>>>>>>>>>>>>>
>>>>>>>>>>>>> 24h/day.
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I updated the KIP today with some points worth talking about,
>>>>>>>>>>>>>>
>>>>>>>>>>>>> should
>>>>>>>>>
>>>>>>>>>> anyone
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> be so inclined to check it out. Currently we are running this
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> code
>>>>>>>>
>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>>> production to handle relational joins from our Kafka Connect
>>>>>>>>>>>>>>> topics, as
>>>>>>>>>>>>>>> per
>>>>>>>>>>>>>>> the original motivation of the KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As for your question regarding GraphNodes, it is for
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> extending
>>>>>>>
>>>>>>>> Streams
>>>>>>>>>>>>>>>> optimization framework. You can find more details on
>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The main idea is that instead of directly building up the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "physical
>>>>>>>>>
>>>>>>>>>> topology" (represented as Topology in the public package, and
>>>>>>>>>>>>>>>> internally
>>>>>>>>>>>>>>>> built as the ProcessorTopology class) while users are
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> specifying
>>>>>>>>
>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>>> transformation operators, we first keep it as a "logical
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> topology"
>>>>>>>>>
>>>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> then
>>>>>>>>>
>>>>>>>>>> only
>>>>>>>>>>>>>>>> execute the optimization and the construction of the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "physical"
>>>>>>>
>>>>>>>> Topology
>>>>>>>>>>>>>>>> when StreamsBuilder.build() is called.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Back to your question, I think it makes more sense to add a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> new
>>>>>>>
>>>>>>>> type of
>>>>>>>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
>>>>>>>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> will
>>>>>>>>>
>>>>>>>>>> have
>>>>>>>>>>>>>>>> multiple connected ProcessorNodes to represent a
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (foreign-key)
>>>>>>>
>>>>>>>> join, we
>>>>>>>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> couple of
>>>>>>>>>>>>>>>> them in the logical representation so that in the future we
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> can
>>>>>>>
>>>>>>>> construct
>>>>>>>>>>>>>>>> the physical topology differently (e.g. having another way
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> than
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> current
>>>>>>>>>>>>>>>> distributed hash-join).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -------------------------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Back to your questions to KIP-213, I think Jan has
>>>>>>>>>>>>>>>> summarized
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> it
>>>>>>>>
>>>>>>>>> pretty-well. Note that back then we do not have headers
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> support
>>>>>>>
>>>>>>>> so
>>>>>>>>>
>>>>>>>>>> we
>>>>>>>>>>
>>>>>>>>>>> have
>>>>>>>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
>>>>>>>>>>>>>>>> Filipiak<Ja...@trivago.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I love how you are on to this already! I resolve this by
>>>>>>>>>>>>>>>>> "key-widening"
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> treat the result of FKA,and FKB differently.
>>>>>>>>>>>>>>>>> As you can see the output of my join has a Combined Key and
>>>>>>>>>>>>>>>>> therefore I
>>>>>>>>>>>>>>>>> can resolve the "race condition" in a group by
>>>>>>>>>>>>>>>>> if I so desire.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think this reflects more what happens under the hood and
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> makes
>>>>>>>>
>>>>>>>>> it more
>>>>>>>>>>>>>>>>> clear to the user what is going on. The Idea
>>>>>>>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> from
>>>>>>>>
>>>>>>>>> my POV
>>>>>>>>>>>>>>>>> unideal.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> To write into your example:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> key + A, null)
>>>>>>>>>>>>>>>>> (key +B, <joined On FK =B>)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> is what my output would look like.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hope that makes sense :D
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If you do not use headers or other metadata, how do you
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> ensure
>>>>>>>>
>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
>>>>>>>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> you
>>>>>>>
>>>>>>>> need to
>>>>>>>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (FK=B).
>>>>>>>>
>>>>>>>>> In my
>>>>>>>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> for
>>>>>>>>
>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>>> final
>>>>>>>>>>>>>>>>>> output to be in either order - the correctly updated
>>>>>>>>>>>>>>>>>> joined
>>>>>>>>>>>>>>>>>> value, or
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> null for the delete.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I looked back through your code and through the discussion
>>>>>>>>>>>>>>>>>> threads, and
>>>>>>>>>>>>>>>>>> didn't see any information on how you resolved this. I
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> have a
>>>>>>>
>>>>>>>> version
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>> code working for 2.0, I am just adding more integration
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> tests
>>>>>>>
>>>>>>>> and will
>>>>>>>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> on
>>>>>>>
>>>>>>>> resolving
>>>>>>>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
>>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Happy to see that you want to make an effort here.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> not
>>>>>>>
>>>>>>>> rewrite
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> joiners + the merger.
>>>>>>>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> if
>>>>>>>
>>>>>>>> repartition
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> is optimized in 2.0 now.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I made this
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 241+
>>>>>>>
>>>>>>>> KTable+repartition+with+compacted+Topics
>>>>>>>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> combination.
>>>>>>>>>>
>>>>>>>>>>> For us it is vital as it minimized the size we had in our
>>>>>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> message.
>>>>>>>>>>
>>>>>>>>>>> I know about this new  "delete once consumer has read it".
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I
>>>>>>>>
>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> but I
>>>>>>>
>>>>>>>> am
>>>>>>>>>
>>>>>>>>>> certain
>>>>>>>>>>>>>>>>>>> that you do not need RecordHeaders
>>>>>>>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> belong
>>>>>>>>
>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> or to the user, having a mixed use is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> don't
>>>>>>>>
>>>>>>>>> think
>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>> needs more.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> couldn't
>>>>>>>>>>
>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> this
>>>>>>>>
>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>> help me see the good sides a little bit more.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> If there is anything unclear with all the text that has
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> been
>>>>>>>
>>>>>>>> written,
>>>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
>>>>>>>>>>>>>>>>>>> the mailing list.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me
>>>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>> previously.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> major
>>>>>>>>>
>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> about
>>>>>>>
>>>>>>>> this:
>>>>>>>>>>
>>>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> node
>>>>>>>>>
>>>>>>>>>> type?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> would it be advisable to construct it from existing
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> GraphNode
>>>>>>>
>>>>>>>> components?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
As I was looking more into RocksDB TTL, I see that we currently do not
support it in Kafka Streams due to a number of technical reasons. As I
don't think that I will be tackling that JIRA at the moment, the current
implementation is indeed unbounded in the highwater table growth.

An alternate option may be to replace the highwater mark table with a
groupBy and then perform a reduce/aggregate. My main concern here is that
technically we could have an unbounded amount of data to group together by
key, and the grouped size could exceed the Kafka maximum record size. When
I built the highwater mark table my intent was to work around this
possibility, as each record is evaluated independently and record sizing
issues do not come into play. If I am incorrect in this assumption, please
correct me, because I am a bit fuzzy on exactly how the groupBy currently
works.

Any thoughts on this are appreciated. I will revisit it again when I have a
bit more time.

Thanks



On Mon, Sep 3, 2018 at 4:55 PM, Adam Bellemare <ad...@gmail.com>
wrote:

> Hi Jan
>
> Thank you for taking the time to look into my PR. I have updated it
> accordingly along with the suggestions from John. Please note that I am by
> no means an expert on Java, so I really do appreciate any Java-specific
> feedback you may have. Do not worry about being overly verbose on it.
>
> You are correct with regards to the highwater mark growing unbounded. One
> option would be to implement the rocksDB TTL to expire records. I am open
> to other options as well.
>
> I have tried to detail the reasoning behind it in the KIP - I have added
> additional comments and I hope that it is clearer now.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 213+Support+non-key+joining+in+KTable#KIP-213Supportnon-
> keyjoininginKTable-MultipleRapidForeign-KeyValueChanges-
> Whyahighwatertableisrequired.
>
> Please keep in mind that there may be something about ordering guarantees
> that I am not aware of. As far as I know, once you begin to operate on
> events in parallel across different nodes within the processor API, there
> are no ordering guarantees and everything is simple first-come,
> first-served(processed). If this is not the case then I am unaware of that
> fact.
>
>
>
> Thanks
>
> Adam
>
>
>
>
>
> On Mon, Sep 3, 2018 at 8:38 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Finished my deeper scan on your approach.
>> Most of the comments I put at the PR are minor code style things.
>> One forward call seems to be a bug though, would be great if you could
>> double check.
>>
>> the one problem I see is that the high watermark store grows unbounded.
>> A key being deleted from the source table does not lead to deletion in
>> the watermark store.
>>
>> I also don't quite grasp the concept why it's needed.  I think the whole
>> offset part can go away?
>> It seems to deal with node failures of some kind but everything should
>> turn out okay without it?
>>
>> Best Jan
>>
>>
>> On 01.09.2018 20:44, Guozhang Wang wrote:
>>
>>> Yes Adam, that makes sense.
>>>
>>> I think it may be better to have a working PR to review before we
>>> complete
>>> the VOTE thread. In my previous experience a large feature like this are
>>> mostly definitely going to miss some devils in the details in the design
>>> and wiki discussion phases.
>>>
>>> That would unfortunately mean that your implementations may need to be
>>> modified / updated along with the review and further KIP discussion. I
>>> can
>>> understand this can be painful, but that may be the best option we can do
>>> to avoid as much work to be wasted as possible.
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <
>>> adam.bellemare@gmail.com>
>>> wrote:
>>>
>>> Hi Guozhang
>>>>
>>>> By workflow I mean just the overall process of how the KIP is
>>>> implemented.
>>>> Any ideas on the ways to reduce the topic count, materializations, if
>>>> there
>>>> is a better way to resolve out-of-order than a highwater mark table, if
>>>> the
>>>> design philosophy of “keep everything encapsulated within the join
>>>> function” is appropriate, etc. I can implement the changes that John
>>>> suggested, but if my overall workflow is not acceptable I would rather
>>>> address that before making minor changes.
>>>>
>>>> If this requires a full candidate PR ready to go to prod then I can make
>>>> those changes. Hope that clears things up.
>>>>
>>>> Thanks
>>>>
>>>> Adam
>>>>
>>>> On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com> wrote:
>>>>>
>>>>> Hi Adam,
>>>>>
>>>>> What do you mean by "additional comments on the workflow.", do you mean
>>>>>
>>>> to
>>>>
>>>>> let other review your PR https://github.com/apache/kafka/pull/5527 ?
>>>>> Is
>>>>>
>>>> is
>>>>
>>>>> ready for reviews?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>> On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
>>>>>
>>>> adam.bellemare@gmail.com>
>>>>
>>>>> wrote:
>>>>>
>>>>> Okay, I will implement John's suggestion of namespacing the external
>>>>>> headers prior to processing, and then removing the namespacing prior
>>>>>> to
>>>>>> emitting. A potential future KIP could be to provide this namespacing
>>>>>> automatically.
>>>>>>
>>>>>> I would also appreciate any other additional comments on the workflow.
>>>>>>
>>>>> My
>>>>
>>>>> goal is suss out agreement prior to moving to a vote.
>>>>>>
>>>>>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>>
>>>>>> wrote:
>>>>
>>>>> I like John's idea as well: for this KIP specifically as we do not
>>>>>>>
>>>>>> expect
>>>>
>>>>> any other consumers to read the repartition topics externally, we can
>>>>>>> slightly prefix the header to be safe, while keeping the additional
>>>>>>>
>>>>>> cost
>>>>
>>>>> (note the header field is per-record, so any additional byte is
>>>>>>>
>>>>>> per-record
>>>>>>
>>>>>>> as well) low.
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
>>>>>>>
>>>>>> adam.bellemare@gmail.com
>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi John
>>>>>>>>
>>>>>>>> That is an excellent idea. The header usage I propose would be
>>>>>>>> limited
>>>>>>>> entirely to internal topics, and this could very well be the
>>>>>>>> solution
>>>>>>>>
>>>>>>> to
>>>>>>
>>>>>>> potential conflicts. If we do not officially reserve a prefix "__"
>>>>>>>>
>>>>>>> then I
>>>>>>
>>>>>>> think this would be the safest idea, as it would entirely avoid any
>>>>>>>> accidents (perhaps if a company is using its own "__" prefix for
>>>>>>>> other
>>>>>>>> reasons).
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>>
>>>>>>>> Adam
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
>>>>>>>>
>>>>>>> wrote:
>>>>>>
>>>>>>> Just a quick thought regarding headers:
>>>>>>>>>
>>>>>>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
>>>>>>>>>>
>>>>>>>>> can
>>>>>>
>>>>>>> still
>>>>>>>>>
>>>>>>>>>> consider using some name patterns to reduce the likelihood as much
>>>>>>>>>>
>>>>>>>>> as
>>>>>>
>>>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
>>>>>>>>>> "__internal_[name]"?
>>>>>>>>>>
>>>>>>>>> I think there is a safe way to avoid conflicts, since these headers
>>>>>>>>>
>>>>>>>> are
>>>>>>
>>>>>>> only needed in internal topics (I think):
>>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>>
>>>>>>>>> This is a lot of characters, so we could use a sigil instead (e.g.,
>>>>>>>>>
>>>>>>>> "_"
>>>>>>
>>>>>>> for
>>>>>>>>
>>>>>>>>> internal, "~" for external)
>>>>>>>>>
>>>>>>>>> We simply apply the namespacing when we read user headers from
>>>>>>>>>
>>>>>>>> external
>>>>>>
>>>>>>> topics into the topology and then de-namespace them before we emit
>>>>>>>>>
>>>>>>>> them
>>>>>>
>>>>>>> to
>>>>>>>>
>>>>>>>>> an external topic (via "to" or "through").
>>>>>>>>> Now, it is not possible to collide with user-defined headers.
>>>>>>>>>
>>>>>>>>> That said, I'd also be fine with just reserving "__" as a header
>>>>>>>>>
>>>>>>>> prefix
>>>>>>
>>>>>>> and
>>>>>>>>
>>>>>>>>> not worrying about collisions.
>>>>>>>>>
>>>>>>>>> Thanks for the KIP,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
>>>>>>>>>
>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>
>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Still havent completly grabbed it.
>>>>>>>>>> sorry will read more
>>>>>>>>>>
>>>>>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
>>>>>>>>>>> Cool stuff.
>>>>>>>>>>>
>>>>>>>>>>> I made some random remarks. Did not touch the core of the
>>>>>>>>>>>
>>>>>>>>>> algorithm
>>>>>>
>>>>>>> yet.
>>>>>>>>>
>>>>>>>>>> Will do Monday 100%
>>>>>>>>>>>
>>>>>>>>>>> I don't see Interactive Queries :) like that!
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
>>>>>>>>>>>> I have submitted a PR with my code against trunk:
>>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>>
>>>>>>>>>>>> Do I continue on this thread or do we begin a new one for
>>>>>>>>>>>>
>>>>>>>>>>> discussion?
>>>>>>>>
>>>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
>>>>>>>>>>>>
>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> even before message headers, the option for me always existed
>>>>>>>>>>>>>
>>>>>>>>>>>> to
>>>>>>
>>>>>>> just wrap
>>>>>>>>>>>>> the messages into my own custom envelop.
>>>>>>>>>>>>> So I of course thought this through. One sentence in your last
>>>>>>>>>>>>>
>>>>>>>>>>>> email
>>>>>>>>
>>>>>>>>> triggered all the thought process I put in the back then
>>>>>>>>>>>>> again to design it in the, what i think is the "kafka-way". It
>>>>>>>>>>>>>
>>>>>>>>>>>> ended
>>>>>>>>
>>>>>>>>> up
>>>>>>>>>
>>>>>>>>>> ranting a little about what happened in the past.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I see plenty of colleagues of mine falling into traps in the
>>>>>>>>>>>>>
>>>>>>>>>>>> API,
>>>>>>
>>>>>>> that I
>>>>>>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
>>>>>>>>>>>>> feeling again. So I hope it gives you some insights into my
>>>>>>>>>>>>>
>>>>>>>>>>>> though
>>>>>>>
>>>>>>>> process. I am aware that since i never ported 213 to higher
>>>>>>>>>>>>> streams version, I don't really have a steak here and
>>>>>>>>>>>>>
>>>>>>>>>>>> initially I
>>>>>>
>>>>>>> didn't
>>>>>>>>>>>>> feel like actually sending it. But maybe you can pull
>>>>>>>>>>>>> something good from it.
>>>>>>>>>>>>>
>>>>>>>>>>>>>   Best jan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow includes
>>>>>>>>>>>>>> information
>>>>>>>>>>>>>> about which record is processed first? I understand about a
>>>>>>>>>>>>>> CombinedKey
>>>>>>>>>>>>>> with both the Foreign and Primary key, but I don't see how you
>>>>>>>>>>>>>>
>>>>>>>>>>>>> track
>>>>>>>>
>>>>>>>>> ordering metadata in there unless you actually included a
>>>>>>>>>>>>>>
>>>>>>>>>>>>> metadata
>>>>>>>
>>>>>>>> field
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> the key type as well.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
>>>>>>>>>>>>>>
>>>>>>>>>>>>> be
>>>>>>
>>>>>>> used in
>>>>>>>>>>>>>> just the user-space? It seems that it is possible that a
>>>>>>>>>>>>>>
>>>>>>>>>>>>> collision
>>>>>>>
>>>>>>>> on the
>>>>>>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
>>>>>>>>>>>>>>
>>>>>>>>>>>>> instance,
>>>>>>
>>>>>>> if
>>>>>>>>
>>>>>>>>> I
>>>>>>>>>
>>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> user
>>>>>>>>
>>>>>>>>> already
>>>>>>>>>>>>>> specified their own header with the same key name, then it
>>>>>>>>>>>>>>
>>>>>>>>>>>>> appears
>>>>>>>
>>>>>>>> there
>>>>>>>>>>>>>> would be a collision. (This is one of the issues I brought up
>>>>>>>>>>>>>>
>>>>>>>>>>>>> in
>>>>>>
>>>>>>> the KIP).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --------------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I will be posting a prototype PR against trunk within the next
>>>>>>>>>>>>>>
>>>>>>>>>>>>> day
>>>>>>>
>>>>>>>> or two.
>>>>>>>>>>>>>> One thing I need to point out is that my design very strictly
>>>>>>>>>>>>>>
>>>>>>>>>>>>> wraps
>>>>>>>>
>>>>>>>>> the
>>>>>>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
>>>>>>>>>>>>>>
>>>>>>>>>>>>> function.
>>>>>>
>>>>>>> There is
>>>>>>>>>>>>>> no
>>>>>>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
>>>>>>>>>>>>>>
>>>>>>>>>>>>> with
>>>>>>>
>>>>>>>> regards
>>>>>>>>>>>>>> to out-of-order processing and no need for the DSL user to
>>>>>>>>>>>>>>
>>>>>>>>>>>>> even
>>>>>>
>>>>>>> know
>>>>>>>>
>>>>>>>>> what's
>>>>>>>>>>>>>> going on inside of the function. The code simply returns the
>>>>>>>>>>>>>> results of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> join, keyed by the original key. Currently my API mirrors
>>>>>>>>>>>>>> identically the
>>>>>>>>>>>>>> format of the data returned by the regular join function, and
>>>>>>>>>>>>>>
>>>>>>>>>>>>> I
>>>>>>
>>>>>>> believe
>>>>>>>>>>>>>> that this is very useful to many users of the DSL. It is my
>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>> that one of the main design goals of the DSL is to provide
>>>>>>>>>>>>>>
>>>>>>>>>>>>> higher
>>>>>>>
>>>>>>>> level
>>>>>>>>>>>>>> functionality without requiring the users to know exactly
>>>>>>>>>>>>>>
>>>>>>>>>>>>> what's
>>>>>>
>>>>>>> going on
>>>>>>>>>>>>>> under the hood. With this in mind, I thought it best to solve
>>>>>>>>>>>>>> ordering and
>>>>>>>>>>>>>> partitioning problems within the function and eliminate the
>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>> users to do additional work after the fact to resolve the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> results
>>>>>>>
>>>>>>>> of their
>>>>>>>>>>>>>> join. Basically, I am assuming that most users of the DSL just
>>>>>>>>>>>>>> "want it to
>>>>>>>>>>>>>> work" and want it to be easy. I did this operating under the
>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>> that if a user truly wants to optimize their own workflow down
>>>>>>>>>>>>>>
>>>>>>>>>>>>> to
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> finest details then they will break from strictly using the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> DSL
>>>>>>
>>>>>>> and
>>>>>>>>
>>>>>>>>> move
>>>>>>>>>>>>>> down to the processors API.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think. The abstraction is not powerful enough
>>>>>>>>>>>>> to not have kafka specifics leak up The leak I currently think
>>>>>>>>>>>>>
>>>>>>>>>>>> this
>>>>>>>
>>>>>>>> has is
>>>>>>>>>>>>> that you can not reliable prevent the delete coming out first,
>>>>>>>>>>>>> before you emit the correct new record. As it is an abstraction
>>>>>>>>>>>>> entirely
>>>>>>>>>>>>> around kafka.
>>>>>>>>>>>>> I can only recommend to not to. Honesty and simplicity should
>>>>>>>>>>>>>
>>>>>>>>>>>> always
>>>>>>>>
>>>>>>>>> be
>>>>>>>>>
>>>>>>>>>> first prio
>>>>>>>>>>>>> trying to hide this just makes it more complex, less
>>>>>>>>>>>>>
>>>>>>>>>>>> understandable
>>>>>>>
>>>>>>>> and
>>>>>>>>>
>>>>>>>>>> will lead to mistakes
>>>>>>>>>>>>> in usage.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
>>>>>>>>>>>>> optimization stages.
>>>>>>>>>>>>> Can someone give me an example of an optimisation that really
>>>>>>>>>>>>>
>>>>>>>>>>>> can't
>>>>>>>
>>>>>>>> be
>>>>>>>>>
>>>>>>>>>> handled by the user
>>>>>>>>>>>>> constructing his topology differently?
>>>>>>>>>>>>> Having reusable Processor API components accessible by the DSL
>>>>>>>>>>>>>
>>>>>>>>>>>> and
>>>>>>>
>>>>>>>> composable as
>>>>>>>>>>>>> one likes is exactly where DSL should max out and KSQL should
>>>>>>>>>>>>>
>>>>>>>>>>>> do
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> next
>>>>>>>>>>>>> step.
>>>>>>>>>>>>> I find it very unprofessional from a software engineering
>>>>>>>>>>>>>
>>>>>>>>>>>> approach
>>>>>>>
>>>>>>>> to run
>>>>>>>>>>>>> software where
>>>>>>>>>>>>> you can not at least senseful reason about the inner workings
>>>>>>>>>>>>>
>>>>>>>>>>>> of
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> libraries used.
>>>>>>>>>>>>> Gives this people have to read and understand in anyway, why
>>>>>>>>>>>>>
>>>>>>>>>>>> try
>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>>> hide
>>>>>>>>>>>>> it?
>>>>>>>>>>>>>
>>>>>>>>>>>>> It really miss the beauty of 0.10 version DSL.
>>>>>>>>>>>>> Apparently not a thing I can influence but just warn about.
>>>>>>>>>>>>>
>>>>>>>>>>>>> @gouzhang
>>>>>>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
>>>>>>>>>>>>>
>>>>>>>>>>>> prune
>>>>>>>
>>>>>>>> from
>>>>>>>>>
>>>>>>>>>> stream app's
>>>>>>>>>>>>> because people just keep passing Materialized's into all the
>>>>>>>>>>>>> operations.
>>>>>>>>>>>>> :D :'-(
>>>>>>>>>>>>> I regret that I couldn't convince you guys back then. Plus this
>>>>>>>>>>>>>
>>>>>>>>>>>> whole
>>>>>>>>
>>>>>>>>> entire topology as a floating
>>>>>>>>>>>>> interface chain, never seen it anywhere :-/ :'(
>>>>>>>>>>>>>
>>>>>>>>>>>>> I don't know. I guess this is just me regretting to only have
>>>>>>>>>>>>>
>>>>>>>>>>>> 24h/day.
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I updated the KIP today with some points worth talking about,
>>>>>>>>>>>>>
>>>>>>>>>>>> should
>>>>>>>>
>>>>>>>>> anyone
>>>>>>>>>>>>>
>>>>>>>>>>>>>> be so inclined to check it out. Currently we are running this
>>>>>>>>>>>>>>
>>>>>>>>>>>>> code
>>>>>>>
>>>>>>>> in
>>>>>>>>>
>>>>>>>>>> production to handle relational joins from our Kafka Connect
>>>>>>>>>>>>>> topics, as
>>>>>>>>>>>>>> per
>>>>>>>>>>>>>> the original motivation of the KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
>>>>>>>>>>>>>>
>>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As for your question regarding GraphNodes, it is for
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> extending
>>>>>>
>>>>>>> Streams
>>>>>>>>>>>>>>> optimization framework. You can find more details on
>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The main idea is that instead of directly building up the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> "physical
>>>>>>>>
>>>>>>>>> topology" (represented as Topology in the public package, and
>>>>>>>>>>>>>>> internally
>>>>>>>>>>>>>>> built as the ProcessorTopology class) while users are
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> specifying
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> transformation operators, we first keep it as a "logical
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> topology"
>>>>>>>>
>>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> then
>>>>>>>>
>>>>>>>>> only
>>>>>>>>>>>>>>> execute the optimization and the construction of the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> "physical"
>>>>>>
>>>>>>> Topology
>>>>>>>>>>>>>>> when StreamsBuilder.build() is called.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Back to your question, I think it makes more sense to add a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> new
>>>>>>
>>>>>>> type of
>>>>>>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
>>>>>>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology we
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> will
>>>>>>>>
>>>>>>>>> have
>>>>>>>>>>>>>>> multiple connected ProcessorNodes to represent a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (foreign-key)
>>>>>>
>>>>>>> join, we
>>>>>>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just a
>>>>>>>>>>>>>>> couple of
>>>>>>>>>>>>>>> them in the logical representation so that in the future we
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> can
>>>>>>
>>>>>>> construct
>>>>>>>>>>>>>>> the physical topology differently (e.g. having another way
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> than
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> current
>>>>>>>>>>>>>>> distributed hash-join).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -------------------------------------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Back to your questions to KIP-213, I think Jan has summarized
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> it
>>>>>>>
>>>>>>>> pretty-well. Note that back then we do not have headers
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> support
>>>>>>
>>>>>>> so
>>>>>>>>
>>>>>>>>> we
>>>>>>>>>
>>>>>>>>>> have
>>>>>>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
>>>>>>>>>>>>>>> Filipiak<Ja...@trivago.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I love how you are on to this already! I resolve this by
>>>>>>>>>>>>>>>> "key-widening"
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> treat the result of FKA,and FKB differently.
>>>>>>>>>>>>>>>> As you can see the output of my join has a Combined Key and
>>>>>>>>>>>>>>>> therefore I
>>>>>>>>>>>>>>>> can resolve the "race condition" in a group by
>>>>>>>>>>>>>>>> if I so desire.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think this reflects more what happens under the hood and
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> makes
>>>>>>>
>>>>>>>> it more
>>>>>>>>>>>>>>>> clear to the user what is going on. The Idea
>>>>>>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> from
>>>>>>>
>>>>>>>> my POV
>>>>>>>>>>>>>>>> unideal.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> To write into your example:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> key + A, null)
>>>>>>>>>>>>>>>> (key +B, <joined On FK =B>)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> is what my output would look like.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hope that makes sense :D
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If you do not use headers or other metadata, how do you
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ensure
>>>>>>>
>>>>>>>> that
>>>>>>>>>
>>>>>>>>>> changes
>>>>>>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
>>>>>>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> you
>>>>>>
>>>>>>> need to
>>>>>>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> (FK=B).
>>>>>>>
>>>>>>>> In my
>>>>>>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> for
>>>>>>>
>>>>>>>> the
>>>>>>>>>
>>>>>>>>>> final
>>>>>>>>>>>>>>>>> output to be in either order - the correctly updated joined
>>>>>>>>>>>>>>>>> value, or
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> null for the delete.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I looked back through your code and through the discussion
>>>>>>>>>>>>>>>>> threads, and
>>>>>>>>>>>>>>>>> didn't see any information on how you resolved this. I
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> have a
>>>>>>
>>>>>>> version
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>> code working for 2.0, I am just adding more integration
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> tests
>>>>>>
>>>>>>> and will
>>>>>>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> on
>>>>>>
>>>>>>> resolving
>>>>>>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
>>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Happy to see that you want to make an effort here.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> not
>>>>>>
>>>>>>> rewrite
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> joiners + the merger.
>>>>>>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> if
>>>>>>
>>>>>>> repartition
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> is optimized in 2.0 now.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I made this
>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 241+
>>>>>>
>>>>>>> KTable+repartition+with+compacted+Topics
>>>>>>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> combination.
>>>>>>>>>
>>>>>>>>>> For us it is vital as it minimized the size we had in our
>>>>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> message.
>>>>>>>>>
>>>>>>>>>> I know about this new  "delete once consumer has read it".
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I
>>>>>>>
>>>>>>>> don't
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> but I
>>>>>>
>>>>>>> am
>>>>>>>>
>>>>>>>>> certain
>>>>>>>>>>>>>>>>>> that you do not need RecordHeaders
>>>>>>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> belong
>>>>>>>
>>>>>>>> to the
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> or to the user, having a mixed use is
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> don't
>>>>>>>
>>>>>>>> think
>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>> needs more.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> couldn't
>>>>>>>>>
>>>>>>>>>> never
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> this
>>>>>>>
>>>>>>>> can
>>>>>>>>>
>>>>>>>>>> also
>>>>>>>>>>>>>>>>>> help me see the good sides a little bit more.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If there is anything unclear with all the text that has
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> been
>>>>>>
>>>>>>> written,
>>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
>>>>>>>>>>>>>>>>>> the mailing list.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me directly
>>>>>>>>>>>>>>>>>> previously.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> major
>>>>>>>>
>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> about
>>>>>>
>>>>>>> this:
>>>>>>>>>
>>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> node
>>>>>>>>
>>>>>>>>> type?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> would it be advisable to construct it from existing
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> GraphNode
>>>>>>
>>>>>>> components?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Adam Bellemare <ad...@gmail.com>.
Hi Jan

Thank you for taking the time to look into my PR. I have updated it
accordingly along with the suggestions from John. Please note that I am by
no means an expert on Java, so I really do appreciate any Java-specific
feedback you may have. Do not worry about being overly verbose on it.

You are correct with regards to the highwater mark growing unbounded. One
option would be to implement the rocksDB TTL to expire records. I am open
to other options as well.

I have tried to detail the reasoning behind it in the KIP - I have added
additional comments and I hope that it is clearer now.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable#KIP-213Supportnon-keyjoininginKTable-MultipleRapidForeign-KeyValueChanges-Whyahighwatertableisrequired
.

Please keep in mind that there may be something about ordering guarantees
that I am not aware of. As far as I know, once you begin to operate on
events in parallel across different nodes within the processor API, there
are no ordering guarantees and everything is simple first-come,
first-served(processed). If this is not the case then I am unaware of that
fact.



Thanks

Adam





On Mon, Sep 3, 2018 at 8:38 AM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Finished my deeper scan on your approach.
> Most of the comments I put at the PR are minor code style things.
> One forward call seems to be a bug though, would be great if you could
> double check.
>
> the one problem I see is that the high watermark store grows unbounded.
> A key being deleted from the source table does not lead to deletion in the
> watermark store.
>
> I also don't quite grasp the concept why it's needed.  I think the whole
> offset part can go away?
> It seems to deal with node failures of some kind but everything should
> turn out okay without it?
>
> Best Jan
>
>
> On 01.09.2018 20:44, Guozhang Wang wrote:
>
>> Yes Adam, that makes sense.
>>
>> I think it may be better to have a working PR to review before we complete
>> the VOTE thread. In my previous experience a large feature like this are
>> mostly definitely going to miss some devils in the details in the design
>> and wiki discussion phases.
>>
>> That would unfortunately mean that your implementations may need to be
>> modified / updated along with the review and further KIP discussion. I can
>> understand this can be painful, but that may be the best option we can do
>> to avoid as much work to be wasted as possible.
>>
>>
>> Guozhang
>>
>>
>> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <
>> adam.bellemare@gmail.com>
>> wrote:
>>
>> Hi Guozhang
>>>
>>> By workflow I mean just the overall process of how the KIP is
>>> implemented.
>>> Any ideas on the ways to reduce the topic count, materializations, if
>>> there
>>> is a better way to resolve out-of-order than a highwater mark table, if
>>> the
>>> design philosophy of “keep everything encapsulated within the join
>>> function” is appropriate, etc. I can implement the changes that John
>>> suggested, but if my overall workflow is not acceptable I would rather
>>> address that before making minor changes.
>>>
>>> If this requires a full candidate PR ready to go to prod then I can make
>>> those changes. Hope that clears things up.
>>>
>>> Thanks
>>>
>>> Adam
>>>
>>> On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com> wrote:
>>>>
>>>> Hi Adam,
>>>>
>>>> What do you mean by "additional comments on the workflow.", do you mean
>>>>
>>> to
>>>
>>>> let other review your PR https://github.com/apache/kafka/pull/5527 ? Is
>>>>
>>> is
>>>
>>>> ready for reviews?
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
>>>>
>>> adam.bellemare@gmail.com>
>>>
>>>> wrote:
>>>>
>>>> Okay, I will implement John's suggestion of namespacing the external
>>>>> headers prior to processing, and then removing the namespacing prior to
>>>>> emitting. A potential future KIP could be to provide this namespacing
>>>>> automatically.
>>>>>
>>>>> I would also appreciate any other additional comments on the workflow.
>>>>>
>>>> My
>>>
>>>> goal is suss out agreement prior to moving to a vote.
>>>>>
>>>>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
>>>>>>
>>>>> wrote:
>>>
>>>> I like John's idea as well: for this KIP specifically as we do not
>>>>>>
>>>>> expect
>>>
>>>> any other consumers to read the repartition topics externally, we can
>>>>>> slightly prefix the header to be safe, while keeping the additional
>>>>>>
>>>>> cost
>>>
>>>> (note the header field is per-record, so any additional byte is
>>>>>>
>>>>> per-record
>>>>>
>>>>>> as well) low.
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
>>>>>>
>>>>> adam.bellemare@gmail.com
>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi John
>>>>>>>
>>>>>>> That is an excellent idea. The header usage I propose would be
>>>>>>> limited
>>>>>>> entirely to internal topics, and this could very well be the solution
>>>>>>>
>>>>>> to
>>>>>
>>>>>> potential conflicts. If we do not officially reserve a prefix "__"
>>>>>>>
>>>>>> then I
>>>>>
>>>>>> think this would be the safest idea, as it would entirely avoid any
>>>>>>> accidents (perhaps if a company is using its own "__" prefix for
>>>>>>> other
>>>>>>> reasons).
>>>>>>>
>>>>>>> Thanks
>>>>>>>
>>>>>>> Adam
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
>>>>>>>
>>>>>> wrote:
>>>>>
>>>>>> Just a quick thought regarding headers:
>>>>>>>>
>>>>>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
>>>>>>>>>
>>>>>>>> can
>>>>>
>>>>>> still
>>>>>>>>
>>>>>>>>> consider using some name patterns to reduce the likelihood as much
>>>>>>>>>
>>>>>>>> as
>>>>>
>>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
>>>>>>>>> "__internal_[name]"?
>>>>>>>>>
>>>>>>>> I think there is a safe way to avoid conflicts, since these headers
>>>>>>>>
>>>>>>> are
>>>>>
>>>>>> only needed in internal topics (I think):
>>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>>
>>>>>>>> This is a lot of characters, so we could use a sigil instead (e.g.,
>>>>>>>>
>>>>>>> "_"
>>>>>
>>>>>> for
>>>>>>>
>>>>>>>> internal, "~" for external)
>>>>>>>>
>>>>>>>> We simply apply the namespacing when we read user headers from
>>>>>>>>
>>>>>>> external
>>>>>
>>>>>> topics into the topology and then de-namespace them before we emit
>>>>>>>>
>>>>>>> them
>>>>>
>>>>>> to
>>>>>>>
>>>>>>>> an external topic (via "to" or "through").
>>>>>>>> Now, it is not possible to collide with user-defined headers.
>>>>>>>>
>>>>>>>> That said, I'd also be fine with just reserving "__" as a header
>>>>>>>>
>>>>>>> prefix
>>>>>
>>>>>> and
>>>>>>>
>>>>>>>> not worrying about collisions.
>>>>>>>>
>>>>>>>> Thanks for the KIP,
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
>>>>>>>>
>>>>>>> Jan.Filipiak@trivago.com
>>>>>
>>>>>> wrote:
>>>>>>>>
>>>>>>>> Still havent completly grabbed it.
>>>>>>>>> sorry will read more
>>>>>>>>>
>>>>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
>>>>>>>>>> Cool stuff.
>>>>>>>>>>
>>>>>>>>>> I made some random remarks. Did not touch the core of the
>>>>>>>>>>
>>>>>>>>> algorithm
>>>>>
>>>>>> yet.
>>>>>>>>
>>>>>>>>> Will do Monday 100%
>>>>>>>>>>
>>>>>>>>>> I don't see Interactive Queries :) like that!
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
>>>>>>>>>>> I have submitted a PR with my code against trunk:
>>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>>
>>>>>>>>>>> Do I continue on this thread or do we begin a new one for
>>>>>>>>>>>
>>>>>>>>>> discussion?
>>>>>>>
>>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
>>>>>>>>>>>
>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> even before message headers, the option for me always existed
>>>>>>>>>>>>
>>>>>>>>>>> to
>>>>>
>>>>>> just wrap
>>>>>>>>>>>> the messages into my own custom envelop.
>>>>>>>>>>>> So I of course thought this through. One sentence in your last
>>>>>>>>>>>>
>>>>>>>>>>> email
>>>>>>>
>>>>>>>> triggered all the thought process I put in the back then
>>>>>>>>>>>> again to design it in the, what i think is the "kafka-way". It
>>>>>>>>>>>>
>>>>>>>>>>> ended
>>>>>>>
>>>>>>>> up
>>>>>>>>
>>>>>>>>> ranting a little about what happened in the past.
>>>>>>>>>>>>
>>>>>>>>>>>> I see plenty of colleagues of mine falling into traps in the
>>>>>>>>>>>>
>>>>>>>>>>> API,
>>>>>
>>>>>> that I
>>>>>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
>>>>>>>>>>>> feeling again. So I hope it gives you some insights into my
>>>>>>>>>>>>
>>>>>>>>>>> though
>>>>>>
>>>>>>> process. I am aware that since i never ported 213 to higher
>>>>>>>>>>>> streams version, I don't really have a steak here and
>>>>>>>>>>>>
>>>>>>>>>>> initially I
>>>>>
>>>>>> didn't
>>>>>>>>>>>> feel like actually sending it. But maybe you can pull
>>>>>>>>>>>> something good from it.
>>>>>>>>>>>>
>>>>>>>>>>>>   Best jan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> @Jan
>>>>>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow includes
>>>>>>>>>>>>> information
>>>>>>>>>>>>> about which record is processed first? I understand about a
>>>>>>>>>>>>> CombinedKey
>>>>>>>>>>>>> with both the Foreign and Primary key, but I don't see how you
>>>>>>>>>>>>>
>>>>>>>>>>>> track
>>>>>>>
>>>>>>>> ordering metadata in there unless you actually included a
>>>>>>>>>>>>>
>>>>>>>>>>>> metadata
>>>>>>
>>>>>>> field
>>>>>>>>>>>>> in
>>>>>>>>>>>>> the key type as well.
>>>>>>>>>>>>>
>>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
>>>>>>>>>>>>>
>>>>>>>>>>>> be
>>>>>
>>>>>> used in
>>>>>>>>>>>>> just the user-space? It seems that it is possible that a
>>>>>>>>>>>>>
>>>>>>>>>>>> collision
>>>>>>
>>>>>>> on the
>>>>>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
>>>>>>>>>>>>>
>>>>>>>>>>>> instance,
>>>>>
>>>>>> if
>>>>>>>
>>>>>>>> I
>>>>>>>>
>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
>>>>>>>>>>>>>
>>>>>>>>>>>> user
>>>>>>>
>>>>>>>> already
>>>>>>>>>>>>> specified their own header with the same key name, then it
>>>>>>>>>>>>>
>>>>>>>>>>>> appears
>>>>>>
>>>>>>> there
>>>>>>>>>>>>> would be a collision. (This is one of the issues I brought up
>>>>>>>>>>>>>
>>>>>>>>>>>> in
>>>>>
>>>>>> the KIP).
>>>>>>>>>>>>>
>>>>>>>>>>>>> --------------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>> I will be posting a prototype PR against trunk within the next
>>>>>>>>>>>>>
>>>>>>>>>>>> day
>>>>>>
>>>>>>> or two.
>>>>>>>>>>>>> One thing I need to point out is that my design very strictly
>>>>>>>>>>>>>
>>>>>>>>>>>> wraps
>>>>>>>
>>>>>>>> the
>>>>>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
>>>>>>>>>>>>>
>>>>>>>>>>>> function.
>>>>>
>>>>>> There is
>>>>>>>>>>>>> no
>>>>>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
>>>>>>>>>>>>>
>>>>>>>>>>>> with
>>>>>>
>>>>>>> regards
>>>>>>>>>>>>> to out-of-order processing and no need for the DSL user to
>>>>>>>>>>>>>
>>>>>>>>>>>> even
>>>>>
>>>>>> know
>>>>>>>
>>>>>>>> what's
>>>>>>>>>>>>> going on inside of the function. The code simply returns the
>>>>>>>>>>>>> results of
>>>>>>>>>>>>> the
>>>>>>>>>>>>> join, keyed by the original key. Currently my API mirrors
>>>>>>>>>>>>> identically the
>>>>>>>>>>>>> format of the data returned by the regular join function, and
>>>>>>>>>>>>>
>>>>>>>>>>>> I
>>>>>
>>>>>> believe
>>>>>>>>>>>>> that this is very useful to many users of the DSL. It is my
>>>>>>>>>>>>> understanding
>>>>>>>>>>>>> that one of the main design goals of the DSL is to provide
>>>>>>>>>>>>>
>>>>>>>>>>>> higher
>>>>>>
>>>>>>> level
>>>>>>>>>>>>> functionality without requiring the users to know exactly
>>>>>>>>>>>>>
>>>>>>>>>>>> what's
>>>>>
>>>>>> going on
>>>>>>>>>>>>> under the hood. With this in mind, I thought it best to solve
>>>>>>>>>>>>> ordering and
>>>>>>>>>>>>> partitioning problems within the function and eliminate the
>>>>>>>>>>>>> requirement
>>>>>>>>>>>>> for
>>>>>>>>>>>>> users to do additional work after the fact to resolve the
>>>>>>>>>>>>>
>>>>>>>>>>>> results
>>>>>>
>>>>>>> of their
>>>>>>>>>>>>> join. Basically, I am assuming that most users of the DSL just
>>>>>>>>>>>>> "want it to
>>>>>>>>>>>>> work" and want it to be easy. I did this operating under the
>>>>>>>>>>>>> assumption
>>>>>>>>>>>>> that if a user truly wants to optimize their own workflow down
>>>>>>>>>>>>>
>>>>>>>>>>>> to
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> finest details then they will break from strictly using the
>>>>>>>>>>>>>
>>>>>>>>>>>> DSL
>>>>>
>>>>>> and
>>>>>>>
>>>>>>>> move
>>>>>>>>>>>>> down to the processors API.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I think. The abstraction is not powerful enough
>>>>>>>>>>>> to not have kafka specifics leak up The leak I currently think
>>>>>>>>>>>>
>>>>>>>>>>> this
>>>>>>
>>>>>>> has is
>>>>>>>>>>>> that you can not reliable prevent the delete coming out first,
>>>>>>>>>>>> before you emit the correct new record. As it is an abstraction
>>>>>>>>>>>> entirely
>>>>>>>>>>>> around kafka.
>>>>>>>>>>>> I can only recommend to not to. Honesty and simplicity should
>>>>>>>>>>>>
>>>>>>>>>>> always
>>>>>>>
>>>>>>>> be
>>>>>>>>
>>>>>>>>> first prio
>>>>>>>>>>>> trying to hide this just makes it more complex, less
>>>>>>>>>>>>
>>>>>>>>>>> understandable
>>>>>>
>>>>>>> and
>>>>>>>>
>>>>>>>>> will lead to mistakes
>>>>>>>>>>>> in usage.
>>>>>>>>>>>>
>>>>>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
>>>>>>>>>>>> optimization stages.
>>>>>>>>>>>> Can someone give me an example of an optimisation that really
>>>>>>>>>>>>
>>>>>>>>>>> can't
>>>>>>
>>>>>>> be
>>>>>>>>
>>>>>>>>> handled by the user
>>>>>>>>>>>> constructing his topology differently?
>>>>>>>>>>>> Having reusable Processor API components accessible by the DSL
>>>>>>>>>>>>
>>>>>>>>>>> and
>>>>>>
>>>>>>> composable as
>>>>>>>>>>>> one likes is exactly where DSL should max out and KSQL should
>>>>>>>>>>>>
>>>>>>>>>>> do
>>>>>
>>>>>> the
>>>>>>>
>>>>>>>> next
>>>>>>>>>>>> step.
>>>>>>>>>>>> I find it very unprofessional from a software engineering
>>>>>>>>>>>>
>>>>>>>>>>> approach
>>>>>>
>>>>>>> to run
>>>>>>>>>>>> software where
>>>>>>>>>>>> you can not at least senseful reason about the inner workings
>>>>>>>>>>>>
>>>>>>>>>>> of
>>>>>
>>>>>> the
>>>>>>>
>>>>>>>> libraries used.
>>>>>>>>>>>> Gives this people have to read and understand in anyway, why
>>>>>>>>>>>>
>>>>>>>>>>> try
>>>>>
>>>>>> to
>>>>>>
>>>>>>> hide
>>>>>>>>>>>> it?
>>>>>>>>>>>>
>>>>>>>>>>>> It really miss the beauty of 0.10 version DSL.
>>>>>>>>>>>> Apparently not a thing I can influence but just warn about.
>>>>>>>>>>>>
>>>>>>>>>>>> @gouzhang
>>>>>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
>>>>>>>>>>>>
>>>>>>>>>>> prune
>>>>>>
>>>>>>> from
>>>>>>>>
>>>>>>>>> stream app's
>>>>>>>>>>>> because people just keep passing Materialized's into all the
>>>>>>>>>>>> operations.
>>>>>>>>>>>> :D :'-(
>>>>>>>>>>>> I regret that I couldn't convince you guys back then. Plus this
>>>>>>>>>>>>
>>>>>>>>>>> whole
>>>>>>>
>>>>>>>> entire topology as a floating
>>>>>>>>>>>> interface chain, never seen it anywhere :-/ :'(
>>>>>>>>>>>>
>>>>>>>>>>>> I don't know. I guess this is just me regretting to only have
>>>>>>>>>>>>
>>>>>>>>>>> 24h/day.
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I updated the KIP today with some points worth talking about,
>>>>>>>>>>>>
>>>>>>>>>>> should
>>>>>>>
>>>>>>>> anyone
>>>>>>>>>>>>
>>>>>>>>>>>>> be so inclined to check it out. Currently we are running this
>>>>>>>>>>>>>
>>>>>>>>>>>> code
>>>>>>
>>>>>>> in
>>>>>>>>
>>>>>>>>> production to handle relational joins from our Kafka Connect
>>>>>>>>>>>>> topics, as
>>>>>>>>>>>>> per
>>>>>>>>>>>>> the original motivation of the KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
>>>>>>>>>>>>>
>>>>>>>>>>>> wangguoz@gmail.com
>>>>>>
>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> As for your question regarding GraphNodes, it is for
>>>>>>>>>>>>>>
>>>>>>>>>>>>> extending
>>>>>
>>>>>> Streams
>>>>>>>>>>>>>> optimization framework. You can find more details on
>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The main idea is that instead of directly building up the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> "physical
>>>>>>>
>>>>>>>> topology" (represented as Topology in the public package, and
>>>>>>>>>>>>>> internally
>>>>>>>>>>>>>> built as the ProcessorTopology class) while users are
>>>>>>>>>>>>>>
>>>>>>>>>>>>> specifying
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> transformation operators, we first keep it as a "logical
>>>>>>>>>>>>>>
>>>>>>>>>>>>> topology"
>>>>>>>
>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
>>>>>>>>>>>>>>
>>>>>>>>>>>>> then
>>>>>>>
>>>>>>>> only
>>>>>>>>>>>>>> execute the optimization and the construction of the
>>>>>>>>>>>>>>
>>>>>>>>>>>>> "physical"
>>>>>
>>>>>> Topology
>>>>>>>>>>>>>> when StreamsBuilder.build() is called.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Back to your question, I think it makes more sense to add a
>>>>>>>>>>>>>>
>>>>>>>>>>>>> new
>>>>>
>>>>>> type of
>>>>>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
>>>>>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology we
>>>>>>>>>>>>>>
>>>>>>>>>>>>> will
>>>>>>>
>>>>>>>> have
>>>>>>>>>>>>>> multiple connected ProcessorNodes to represent a
>>>>>>>>>>>>>>
>>>>>>>>>>>>> (foreign-key)
>>>>>
>>>>>> join, we
>>>>>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just a
>>>>>>>>>>>>>> couple of
>>>>>>>>>>>>>> them in the logical representation so that in the future we
>>>>>>>>>>>>>>
>>>>>>>>>>>>> can
>>>>>
>>>>>> construct
>>>>>>>>>>>>>> the physical topology differently (e.g. having another way
>>>>>>>>>>>>>>
>>>>>>>>>>>>> than
>>>>>
>>>>>> the
>>>>>>>
>>>>>>>> current
>>>>>>>>>>>>>> distributed hash-join).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -------------------------------------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Back to your questions to KIP-213, I think Jan has summarized
>>>>>>>>>>>>>>
>>>>>>>>>>>>> it
>>>>>>
>>>>>>> pretty-well. Note that back then we do not have headers
>>>>>>>>>>>>>>
>>>>>>>>>>>>> support
>>>>>
>>>>>> so
>>>>>>>
>>>>>>>> we
>>>>>>>>
>>>>>>>>> have
>>>>>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
>>>>>>>>>>>>>> Filipiak<Ja...@trivago.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I love how you are on to this already! I resolve this by
>>>>>>>>>>>>>>> "key-widening"
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>> treat the result of FKA,and FKB differently.
>>>>>>>>>>>>>>> As you can see the output of my join has a Combined Key and
>>>>>>>>>>>>>>> therefore I
>>>>>>>>>>>>>>> can resolve the "race condition" in a group by
>>>>>>>>>>>>>>> if I so desire.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think this reflects more what happens under the hood and
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> makes
>>>>>>
>>>>>>> it more
>>>>>>>>>>>>>>> clear to the user what is going on. The Idea
>>>>>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> from
>>>>>>
>>>>>>> my POV
>>>>>>>>>>>>>>> unideal.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> To write into your example:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> key + A, null)
>>>>>>>>>>>>>>> (key +B, <joined On FK =B>)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> is what my output would look like.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hope that makes sense :D
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If you do not use headers or other metadata, how do you
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ensure
>>>>>>
>>>>>>> that
>>>>>>>>
>>>>>>>>> changes
>>>>>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
>>>>>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> you
>>>>>
>>>>>> need to
>>>>>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (FK=B).
>>>>>>
>>>>>>> In my
>>>>>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> for
>>>>>>
>>>>>>> the
>>>>>>>>
>>>>>>>>> final
>>>>>>>>>>>>>>>> output to be in either order - the correctly updated joined
>>>>>>>>>>>>>>>> value, or
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> null for the delete.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I looked back through your code and through the discussion
>>>>>>>>>>>>>>>> threads, and
>>>>>>>>>>>>>>>> didn't see any information on how you resolved this. I
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> have a
>>>>>
>>>>>> version
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>> code working for 2.0, I am just adding more integration
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> tests
>>>>>
>>>>>> and will
>>>>>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> on
>>>>>
>>>>>> resolving
>>>>>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
>>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Happy to see that you want to make an effort here.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> not
>>>>>
>>>>>> rewrite
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> joiners + the merger.
>>>>>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> if
>>>>>
>>>>>> repartition
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> is optimized in 2.0 now.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I made this
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 241+
>>>>>
>>>>>> KTable+repartition+with+compacted+Topics
>>>>>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> combination.
>>>>>>>>
>>>>>>>>> For us it is vital as it minimized the size we had in our
>>>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> message.
>>>>>>>>
>>>>>>>>> I know about this new  "delete once consumer has read it".
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I
>>>>>>
>>>>>>> don't
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> but I
>>>>>
>>>>>> am
>>>>>>>
>>>>>>>> certain
>>>>>>>>>>>>>>>>> that you do not need RecordHeaders
>>>>>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> belong
>>>>>>
>>>>>>> to the
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> or to the user, having a mixed use is
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> don't
>>>>>>
>>>>>>> think
>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>> needs more.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> couldn't
>>>>>>>>
>>>>>>>>> never
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> this
>>>>>>
>>>>>>> can
>>>>>>>>
>>>>>>>>> also
>>>>>>>>>>>>>>>>> help me see the good sides a little bit more.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If there is anything unclear with all the text that has
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> been
>>>>>
>>>>>> written,
>>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
>>>>>>>>>>>>>>>>> the mailing list.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me directly
>>>>>>>>>>>>>>>>> previously.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> major
>>>>>>>
>>>>>>>> changes
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> about
>>>>>
>>>>>> this:
>>>>>>>>
>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> node
>>>>>>>
>>>>>>>> type?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> would it be advisable to construct it from existing
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> GraphNode
>>>>>
>>>>>> components?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

Posted by Jan Filipiak <Ja...@trivago.com>.
Finished my deeper scan on your approach.
Most of the comments I put at the PR are minor code style things.
One forward call seems to be a bug though, would be great if you could 
double check.

the one problem I see is that the high watermark store grows unbounded.
A key being deleted from the source table does not lead to deletion in 
the watermark store.

I also don't quite grasp the concept why it's needed.  I think the whole 
offset part can go away?
It seems to deal with node failures of some kind but everything should 
turn out okay without it?

Best Jan


On 01.09.2018 20:44, Guozhang Wang wrote:
> Yes Adam, that makes sense.
>
> I think it may be better to have a working PR to review before we complete
> the VOTE thread. In my previous experience a large feature like this are
> mostly definitely going to miss some devils in the details in the design
> and wiki discussion phases.
>
> That would unfortunately mean that your implementations may need to be
> modified / updated along with the review and further KIP discussion. I can
> understand this can be painful, but that may be the best option we can do
> to avoid as much work to be wasted as possible.
>
>
> Guozhang
>
>
> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare <ad...@gmail.com>
> wrote:
>
>> Hi Guozhang
>>
>> By workflow I mean just the overall process of how the KIP is implemented.
>> Any ideas on the ways to reduce the topic count, materializations, if there
>> is a better way to resolve out-of-order than a highwater mark table, if the
>> design philosophy of “keep everything encapsulated within the join
>> function” is appropriate, etc. I can implement the changes that John
>> suggested, but if my overall workflow is not acceptable I would rather
>> address that before making minor changes.
>>
>> If this requires a full candidate PR ready to go to prod then I can make
>> those changes. Hope that clears things up.
>>
>> Thanks
>>
>> Adam
>>
>>> On Aug 29, 2018, at 12:42 PM, Guozhang Wang <wa...@gmail.com> wrote:
>>>
>>> Hi Adam,
>>>
>>> What do you mean by "additional comments on the workflow.", do you mean
>> to
>>> let other review your PR https://github.com/apache/kafka/pull/5527 ? Is
>> is
>>> ready for reviews?
>>>
>>>
>>> Guozhang
>>>
>>> On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare <
>> adam.bellemare@gmail.com>
>>> wrote:
>>>
>>>> Okay, I will implement John's suggestion of namespacing the external
>>>> headers prior to processing, and then removing the namespacing prior to
>>>> emitting. A potential future KIP could be to provide this namespacing
>>>> automatically.
>>>>
>>>> I would also appreciate any other additional comments on the workflow.
>> My
>>>> goal is suss out agreement prior to moving to a vote.
>>>>
>>>>> On Mon, Aug 27, 2018 at 3:19 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>>> I like John's idea as well: for this KIP specifically as we do not
>> expect
>>>>> any other consumers to read the repartition topics externally, we can
>>>>> slightly prefix the header to be safe, while keeping the additional
>> cost
>>>>> (note the header field is per-record, so any additional byte is
>>>> per-record
>>>>> as well) low.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>> On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare <
>>>> adam.bellemare@gmail.com
>>>>> wrote:
>>>>>
>>>>>> Hi John
>>>>>>
>>>>>> That is an excellent idea. The header usage I propose would be limited
>>>>>> entirely to internal topics, and this could very well be the solution
>>>> to
>>>>>> potential conflicts. If we do not officially reserve a prefix "__"
>>>> then I
>>>>>> think this would be the safest idea, as it would entirely avoid any
>>>>>> accidents (perhaps if a company is using its own "__" prefix for other
>>>>>> reasons).
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>> On Tue, Aug 21, 2018 at 2:24 PM, John Roesler <jo...@confluent.io>
>>>> wrote:
>>>>>>> Just a quick thought regarding headers:
>>>>>>>> I think there is no absolute-safe ways to avoid conflicts, but we
>>>> can
>>>>>>> still
>>>>>>>> consider using some name patterns to reduce the likelihood as much
>>>> as
>>>>>>>> possible.. e.g. consider sth. like the internal topics naming: e.g.
>>>>>>>> "__internal_[name]"?
>>>>>>> I think there is a safe way to avoid conflicts, since these headers
>>>> are
>>>>>>> only needed in internal topics (I think):
>>>>>>> For internal and changelog topics, we can namespace all headers:
>>>>>>> * user-defined headers are namespaced as "external." + headerKey
>>>>>>> * internal headers are namespaced as "internal." + headerKey
>>>>>>>
>>>>>>> This is a lot of characters, so we could use a sigil instead (e.g.,
>>>> "_"
>>>>>> for
>>>>>>> internal, "~" for external)
>>>>>>>
>>>>>>> We simply apply the namespacing when we read user headers from
>>>> external
>>>>>>> topics into the topology and then de-namespace them before we emit
>>>> them
>>>>>> to
>>>>>>> an external topic (via "to" or "through").
>>>>>>> Now, it is not possible to collide with user-defined headers.
>>>>>>>
>>>>>>> That said, I'd also be fine with just reserving "__" as a header
>>>> prefix
>>>>>> and
>>>>>>> not worrying about collisions.
>>>>>>>
>>>>>>> Thanks for the KIP,
>>>>>>> -John
>>>>>>>
>>>>>>> On Tue, Aug 21, 2018 at 9:48 AM Jan Filipiak <
>>>> Jan.Filipiak@trivago.com
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Still havent completly grabbed it.
>>>>>>>> sorry will read more
>>>>>>>>
>>>>>>>>> On 17.08.2018 21:23, Jan Filipiak wrote:
>>>>>>>>> Cool stuff.
>>>>>>>>>
>>>>>>>>> I made some random remarks. Did not touch the core of the
>>>> algorithm
>>>>>>> yet.
>>>>>>>>> Will do Monday 100%
>>>>>>>>>
>>>>>>>>> I don't see Interactive Queries :) like that!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> On 17.08.2018 20:28, Adam Bellemare wrote:
>>>>>>>>>> I have submitted a PR with my code against trunk:
>>>>>>>>>> https://github.com/apache/kafka/pull/5527
>>>>>>>>>>
>>>>>>>>>> Do I continue on this thread or do we begin a new one for
>>>>>> discussion?
>>>>>>>>>> On Thu, Aug 16, 2018 at 1:40 AM, Jan Filipiak <
>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> even before message headers, the option for me always existed
>>>> to
>>>>>>>>>>> just wrap
>>>>>>>>>>> the messages into my own custom envelop.
>>>>>>>>>>> So I of course thought this through. One sentence in your last
>>>>>> email
>>>>>>>>>>> triggered all the thought process I put in the back then
>>>>>>>>>>> again to design it in the, what i think is the "kafka-way". It
>>>>>> ended
>>>>>>> up
>>>>>>>>>>> ranting a little about what happened in the past.
>>>>>>>>>>>
>>>>>>>>>>> I see plenty of colleagues of mine falling into traps in the
>>>> API,
>>>>>>>>>>> that I
>>>>>>>>>>> did warn about in the 1.0 DSL rewrite. I have the same
>>>>>>>>>>> feeling again. So I hope it gives you some insights into my
>>>>> though
>>>>>>>>>>> process. I am aware that since i never ported 213 to higher
>>>>>>>>>>> streams version, I don't really have a steak here and
>>>> initially I
>>>>>>>>>>> didn't
>>>>>>>>>>> feel like actually sending it. But maybe you can pull
>>>>>>>>>>> something good from it.
>>>>>>>>>>>
>>>>>>>>>>>   Best jan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> On 15.08.2018 04:44, Adam Bellemare wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> @Jan
>>>>>>>>>>>> Thanks Jan. I take it you mean "key-widening" somehow includes
>>>>>>>>>>>> information
>>>>>>>>>>>> about which record is processed first? I understand about a
>>>>>>>>>>>> CombinedKey
>>>>>>>>>>>> with both the Foreign and Primary key, but I don't see how you
>>>>>> track
>>>>>>>>>>>> ordering metadata in there unless you actually included a
>>>>> metadata
>>>>>>>>>>>> field
>>>>>>>>>>>> in
>>>>>>>>>>>> the key type as well.
>>>>>>>>>>>>
>>>>>>>>>>>> @Guozhang
>>>>>>>>>>>> As Jan mentioned earlier, is Record Headers mean to strictly
>>>> be
>>>>>>>>>>>> used in
>>>>>>>>>>>> just the user-space? It seems that it is possible that a
>>>>> collision
>>>>>>>>>>>> on the
>>>>>>>>>>>> (key,value) tuple I wish to add to it could occur. For
>>>> instance,
>>>>>> if
>>>>>>> I
>>>>>>>>>>>> wanted to add a ("foreignKeyOffset",10) to the Headers but the
>>>>>> user
>>>>>>>>>>>> already
>>>>>>>>>>>> specified their own header with the same key name, then it
>>>>> appears
>>>>>>>>>>>> there
>>>>>>>>>>>> would be a collision. (This is one of the issues I brought up
>>>> in
>>>>>>>>>>>> the KIP).
>>>>>>>>>>>>
>>>>>>>>>>>> --------------------------------
>>>>>>>>>>>>
>>>>>>>>>>>> I will be posting a prototype PR against trunk within the next
>>>>> day
>>>>>>>>>>>> or two.
>>>>>>>>>>>> One thing I need to point out is that my design very strictly
>>>>>> wraps
>>>>>>>>>>>> the
>>>>>>>>>>>> entire foreignKeyJoin process entirely within the DSL
>>>> function.
>>>>>>>>>>>> There is
>>>>>>>>>>>> no
>>>>>>>>>>>> exposure of CombinedKeys or widened keys, nothing to resolve
>>>>> with
>>>>>>>>>>>> regards
>>>>>>>>>>>> to out-of-order processing and no need for the DSL user to
>>>> even
>>>>>> know
>>>>>>>>>>>> what's
>>>>>>>>>>>> going on inside of the function. The code simply returns the
>>>>>>>>>>>> results of
>>>>>>>>>>>> the
>>>>>>>>>>>> join, keyed by the original key. Currently my API mirrors
>>>>>>>>>>>> identically the
>>>>>>>>>>>> format of the data returned by the regular join function, and
>>>> I
>>>>>>>>>>>> believe
>>>>>>>>>>>> that this is very useful to many users of the DSL. It is my
>>>>>>>>>>>> understanding
>>>>>>>>>>>> that one of the main design goals of the DSL is to provide
>>>>> higher
>>>>>>>>>>>> level
>>>>>>>>>>>> functionality without requiring the users to know exactly
>>>> what's
>>>>>>>>>>>> going on
>>>>>>>>>>>> under the hood. With this in mind, I thought it best to solve
>>>>>>>>>>>> ordering and
>>>>>>>>>>>> partitioning problems within the function and eliminate the
>>>>>>>>>>>> requirement
>>>>>>>>>>>> for
>>>>>>>>>>>> users to do additional work after the fact to resolve the
>>>>> results
>>>>>>>>>>>> of their
>>>>>>>>>>>> join. Basically, I am assuming that most users of the DSL just
>>>>>>>>>>>> "want it to
>>>>>>>>>>>> work" and want it to be easy. I did this operating under the
>>>>>>>>>>>> assumption
>>>>>>>>>>>> that if a user truly wants to optimize their own workflow down
>>>>> to
>>>>>>> the
>>>>>>>>>>>> finest details then they will break from strictly using the
>>>> DSL
>>>>>> and
>>>>>>>>>>>> move
>>>>>>>>>>>> down to the processors API.
>>>>>>>>>>>>
>>>>>>>>>>> I think. The abstraction is not powerful enough
>>>>>>>>>>> to not have kafka specifics leak up The leak I currently think
>>>>> this
>>>>>>>>>>> has is
>>>>>>>>>>> that you can not reliable prevent the delete coming out first,
>>>>>>>>>>> before you emit the correct new record. As it is an abstraction
>>>>>>>>>>> entirely
>>>>>>>>>>> around kafka.
>>>>>>>>>>> I can only recommend to not to. Honesty and simplicity should
>>>>>> always
>>>>>>> be
>>>>>>>>>>> first prio
>>>>>>>>>>> trying to hide this just makes it more complex, less
>>>>> understandable
>>>>>>> and
>>>>>>>>>>> will lead to mistakes
>>>>>>>>>>> in usage.
>>>>>>>>>>>
>>>>>>>>>>> Exactly why I am also in big disfavour of GraphNodes and later
>>>>>>>>>>> optimization stages.
>>>>>>>>>>> Can someone give me an example of an optimisation that really
>>>>> can't
>>>>>>> be
>>>>>>>>>>> handled by the user
>>>>>>>>>>> constructing his topology differently?
>>>>>>>>>>> Having reusable Processor API components accessible by the DSL
>>>>> and
>>>>>>>>>>> composable as
>>>>>>>>>>> one likes is exactly where DSL should max out and KSQL should
>>>> do
>>>>>> the
>>>>>>>>>>> next
>>>>>>>>>>> step.
>>>>>>>>>>> I find it very unprofessional from a software engineering
>>>>> approach
>>>>>>>>>>> to run
>>>>>>>>>>> software where
>>>>>>>>>>> you can not at least senseful reason about the inner workings
>>>> of
>>>>>> the
>>>>>>>>>>> libraries used.
>>>>>>>>>>> Gives this people have to read and understand in anyway, why
>>>> try
>>>>> to
>>>>>>>>>>> hide
>>>>>>>>>>> it?
>>>>>>>>>>>
>>>>>>>>>>> It really miss the beauty of 0.10 version DSL.
>>>>>>>>>>> Apparently not a thing I can influence but just warn about.
>>>>>>>>>>>
>>>>>>>>>>> @gouzhang
>>>>>>>>>>> you can't imagine how many extra IQ-Statestores I constantly
>>>>> prune
>>>>>>> from
>>>>>>>>>>> stream app's
>>>>>>>>>>> because people just keep passing Materialized's into all the
>>>>>>>>>>> operations.
>>>>>>>>>>> :D :'-(
>>>>>>>>>>> I regret that I couldn't convince you guys back then. Plus this
>>>>>> whole
>>>>>>>>>>> entire topology as a floating
>>>>>>>>>>> interface chain, never seen it anywhere :-/ :'(
>>>>>>>>>>>
>>>>>>>>>>> I don't know. I guess this is just me regretting to only have
>>>>>>> 24h/day.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I updated the KIP today with some points worth talking about,
>>>>>> should
>>>>>>>>>>> anyone
>>>>>>>>>>>> be so inclined to check it out. Currently we are running this
>>>>> code
>>>>>>> in
>>>>>>>>>>>> production to handle relational joins from our Kafka Connect
>>>>>>>>>>>> topics, as
>>>>>>>>>>>> per
>>>>>>>>>>>> the original motivation of the KIP.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I believe the foreignKeyJoin should be responsible for. In my
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Aug 14, 2018 at 5:22 PM, Guozhang Wang<
>>>>> wangguoz@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>> As for your question regarding GraphNodes, it is for
>>>> extending
>>>>>>>>>>>>> Streams
>>>>>>>>>>>>> optimization framework. You can find more details on
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6761.
>>>>>>>>>>>>>
>>>>>>>>>>>>> The main idea is that instead of directly building up the
>>>>>> "physical
>>>>>>>>>>>>> topology" (represented as Topology in the public package, and
>>>>>>>>>>>>> internally
>>>>>>>>>>>>> built as the ProcessorTopology class) while users are
>>>>> specifying
>>>>>>> the
>>>>>>>>>>>>> transformation operators, we first keep it as a "logical
>>>>>> topology"
>>>>>>>>>>>>> (represented as GraphNode inside InternalStreamsBuilder). And
>>>>>> then
>>>>>>>>>>>>> only
>>>>>>>>>>>>> execute the optimization and the construction of the
>>>> "physical"
>>>>>>>>>>>>> Topology
>>>>>>>>>>>>> when StreamsBuilder.build() is called.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Back to your question, I think it makes more sense to add a
>>>> new
>>>>>>>>>>>>> type of
>>>>>>>>>>>>> StreamsGraphNode (maybe you can consider inheriting from the
>>>>>>>>>>>>> BaseJoinProcessorNode). Note that although in the Topology we
>>>>>> will
>>>>>>>>>>>>> have
>>>>>>>>>>>>> multiple connected ProcessorNodes to represent a
>>>> (foreign-key)
>>>>>>>>>>>>> join, we
>>>>>>>>>>>>> still want to keep it as a single StreamsGraphNode, or just a
>>>>>>>>>>>>> couple of
>>>>>>>>>>>>> them in the logical representation so that in the future we
>>>> can
>>>>>>>>>>>>> construct
>>>>>>>>>>>>> the physical topology differently (e.g. having another way
>>>> than
>>>>>> the
>>>>>>>>>>>>> current
>>>>>>>>>>>>> distributed hash-join).
>>>>>>>>>>>>>
>>>>>>>>>>>>> -------------------------------------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>> Back to your questions to KIP-213, I think Jan has summarized
>>>>> it
>>>>>>>>>>>>> pretty-well. Note that back then we do not have headers
>>>> support
>>>>>> so
>>>>>>> we
>>>>>>>>>>>>> have
>>>>>>>>>>>>> to do such "key-widening" approach to ensure ordering.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 11:39 PM, Jan
>>>>>>>>>>>>> Filipiak<Ja...@trivago.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Adam,
>>>>>>>>>>>>>> I love how you are on to this already! I resolve this by
>>>>>>>>>>>>>> "key-widening"
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>> treat the result of FKA,and FKB differently.
>>>>>>>>>>>>>> As you can see the output of my join has a Combined Key and
>>>>>>>>>>>>>> therefore I
>>>>>>>>>>>>>> can resolve the "race condition" in a group by
>>>>>>>>>>>>>> if I so desire.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think this reflects more what happens under the hood and
>>>>> makes
>>>>>>>>>>>>>> it more
>>>>>>>>>>>>>> clear to the user what is going on. The Idea
>>>>>>>>>>>>>> of hiding this behind metadata and handle it in the DSL is
>>>>> from
>>>>>>>>>>>>>> my POV
>>>>>>>>>>>>>> unideal.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> To write into your example:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> key + A, null)
>>>>>>>>>>>>>> (key +B, <joined On FK =B>)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> is what my output would look like.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hope that makes sense :D
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 13.08.2018 18:16, Adam Bellemare wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Jan
>>>>>>>>>>>>>>> If you do not use headers or other metadata, how do you
>>>>> ensure
>>>>>>> that
>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>> to the foreign-key value are not resolved out-of-order?
>>>>>>>>>>>>>>> ie: If an event has FK = A, but you change it to FK = B,
>>>> you
>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>> propagate both a delete (FK=A -> null) and an addition
>>>>> (FK=B).
>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>> solution, without maintaining any metadata, it is possible
>>>>> for
>>>>>>> the
>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>>> output to be in either order - the correctly updated joined
>>>>>>>>>>>>>>> value, or
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> null for the delete.
>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (key, <joined On FK =B>)
>>>>>>>>>>>>>>> (key, null)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I looked back through your code and through the discussion
>>>>>>>>>>>>>>> threads, and
>>>>>>>>>>>>>>> didn't see any information on how you resolved this. I
>>>> have a
>>>>>>>>>>>>>>> version
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>> code working for 2.0, I am just adding more integration
>>>> tests
>>>>>>>>>>>>>>> and will
>>>>>>>>>>>>>>> update the KIP accordingly. Any insight you could provide
>>>> on
>>>>>>>>>>>>>>> resolving
>>>>>>>>>>>>>>> out-of-order semantics without metadata would be helpful.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak <
>>>>>>>>>>>>>>> Jan.Filipiak@trivago.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Happy to see that you want to make an effort here.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regarding the ProcessSuppliers I couldn't find a way to
>>>> not
>>>>>>>>>>>>>>>> rewrite
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> joiners + the merger.
>>>>>>>>>>>>>>>> The re-partitioners can be reused in theory. I don't know
>>>> if
>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>> is optimized in 2.0 now.
>>>>>>>>>>>>>>>> I made this
>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>> 241+
>>>>>>>>>>>>>>>> KTable+repartition+with+compacted+Topics
>>>>>>>>>>>>>>>> back then and we are running KIP-213 with KIP-241 in
>>>>>>> combination.
>>>>>>>>>>>>>>>> For us it is vital as it minimized the size we had in our
>>>>>>>>>>>>>>>> repartition
>>>>>>>>>>>>>>>> topics plus it removed the factor of 2 in events on every
>>>>>>> message.
>>>>>>>>>>>>>>>> I know about this new  "delete once consumer has read it".
>>>>> I
>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>> 241 is vital for all usecases, for ours it is. I wanted
>>>>>>>>>>>>>>>> to use 213 to sneak in the foundations for 241 aswell.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don't quite understand what a PropagationWrapper is,
>>>> but I
>>>>>> am
>>>>>>>>>>>>>>>> certain
>>>>>>>>>>>>>>>> that you do not need RecordHeaders
>>>>>>>>>>>>>>>> for 213 and I would try to leave them out. They either
>>>>> belong
>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> DSL
>>>>>>>>>>>>>> or to the user, having a mixed use is
>>>>>>>>>>>>>>>> to be avoided. We run the join with 0.8 logformat and I
>>>>> don't
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>> needs more.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This KIP will be very valuable for the streams project! I
>>>>>>> couldn't
>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>> convince myself to invest into the 1.0+ DSL
>>>>>>>>>>>>>>>> as I used almost all my energy to fight against it. Maybe
>>>>> this
>>>>>>> can
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> help me see the good sides a little bit more.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If there is anything unclear with all the text that has
>>>> been
>>>>>>>>>>>>>>>> written,
>>>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>>>> free to just directly cc me so I don't miss it on
>>>>>>>>>>>>>>>> the mailing list.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best Jan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 08.08.2018 15:26, Adam Bellemare wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> More followup, and +dev as Guozhang replied to me directly
>>>>>>>>>>>>>>>> previously.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I am currently porting the code over to trunk. One of the
>>>>>> major
>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>> since 1.0 is the usage of GraphNodes. I have a question
>>>> about
>>>>>>> this:
>>>>>>>>>>>>>>>>> For a foreignKey joiner, should it have its own dedicated
>>>>>> node
>>>>>>>>>>>>>>>>> type?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>> would it be advisable to construct it from existing
>>>> GraphNode
>>>>>>>>>>>>>>>>> components?
>>>>>>>>>>>>>>>>> For instance, I believe I could construct it from several
>>>>>>>>>>>>>>>>> OptimizableRepartitionNode, some SinkNode, some
>>>> SourceNode,
>>>>>> and
>>>>>>>>>>>>>>>> several
>>>>>>>>>>>>>> StatefulProcessorNode. That being said, there is some
>>>>> underlying
>>>>>>>>>>>>>>>>> complexity
>>>>>>>>>>>>>>>>> to each approach.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I will be switching the KIP-213 to use the RecordHeaders
>>>> in
>>>>>>> Kafka
>>>>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>> instead of the PropagationWrapper, but conceptually it
>>>>> should
>>>>>>>>>>>>>>>>> be the
>>>>>>>>>>>>>>>>> same.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Again, any feedback is welcomed...
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Jul 30, 2018 at 9:38 AM, Adam Bellemare <
>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Guozhang et al
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I was just reading the 2.0 release notes and noticed a
>>>>>> section
>>>>>>> on
>>>>>>>>>>>>>>>>>> Record
>>>>>>>>>>>>>>>>>> Headers.
>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>> 244%3A+Add+Record+Header+support+to+Kafka+Streams+
>>>>>>> Processor+API
>>>>>>>>>>>>>>>>>> I am not yet sure if the contents of a RecordHeader is
>>>>>>>>>>>>>>>>>> propagated
>>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> way through the Sinks and Sources, but if it is, and if
>>>> it
>>>>>>>>>>>>>>>>>> remains
>>>>>>>>>>>>>>>>>> attached
>>>>>>>>>>>>>>>>>> to the record (including null records) I may be able to
>>>>>> ditch
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> propagationWrapper for an implementation using
>>>>> RecordHeader.
>>>>>>>>>>>>>>>>>> I am
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>> sure if this is doable, so if anyone understands
>>>>>> RecordHeader
>>>>>>>>>>>>>>>>>> impl
>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>> than I, I would be happy to hear from you.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> In the meantime, let me know of any questions. I believe
>>>>>> this
>>>>>>>>>>>>>>>>>> PR has
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>>>>> of potential to solve problems for other people, as I
>>>> have
>>>>>>>>>>>>>>>>> encountered
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> number of other companies in the wild all home-brewing
>>>>> their
>>>>>>> own
>>>>>>>>>>>>>>>>>> solutions
>>>>>>>>>>>>>>>>>> to come up with a method of handling relational data in
>>>>>>> streams.
>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Jul 27, 2018 at 1:45 AM, Guozhang
>>>>>>>>>>>>>>>>>> Wang<wa...@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hello Adam,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks for rebooting the discussion of this KIP ! Let me
>>>>>>>>>>>>>>>>>> finish my
>>>>>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>>>>> on the wiki and get back to you soon. Sorry for the
>>>>>> delays..
>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, Jul 24, 2018 at 6:08 AM, Adam Bellemare <
>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Let me kick this off with a few starting points that I
>>>>>>>>>>>>>>>>>>>> would like
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> generate some discussion on.
>>>>>>>>>>>>>>>>>>>> 1) It seems to me that I will need to repartition the
>>>>> data
>>>>>>>>>>>>>>>>>>>> twice -
>>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>> the foreign key, and once back to the primary key. Is
>>>>>> there
>>>>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>> missing here?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2) I believe I will also need to materialize 3 state
>>>>>>>>>>>>>>>>>>>> stores: the
>>>>>>>>>>>>>>>>>>>> prefixScan
>>>>>>>>>>>>>>>>>>>> SS, the highwater mark SS (for out-of-order
>>>> resolution)
>>>>>> and
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> final
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> store, due to the workflow I have laid out. I have not
>>>>>>>>>>>>>>>>>>>> thought of
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>> way yet, but would appreciate any input on this
>>>> matter.
>>>>> I
>>>>>>> have
>>>>>>>>>>>>>>>>>>>> gone
>>>>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>> through the mailing list for the previous discussions
>>>> on
>>>>>>>>>>>>>>>>>>>> this KIP,
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> did not see anything relating to resolving
>>>> out-of-order
>>>>>>>>>>>>>>>>>>>> compute. I
>>>>>>>>>>>>>>>>>>>> cannot
>>>>>>>>>>>>>>>>>>>> see a way around the current three-SS structure that I
>>>>>> have.
>>>>>>>>>>>>>>>>>>>> 3) Caching is disabled on the prefixScan SS, as I do
>>>> not
>>>>>>>>>>>>>>>>>>>> know how
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> resolve the iterator obtained from rocksDB with that of the
>>>>>> cache.
>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>> addition, I must ensure everything is flushed before
>>>> scanning.
>>>>>>>>>>>>>>>>>>> Since
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> materialized prefixScan SS is under "control" of the
>>>>>>>>>>>>>>>>>>>> function, I
>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> anticipate this to be a problem. Performance
>>>> throughput
>>>>>>>>>>>>>>>>>>>> will need
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> tested, but as Jan observed in his initial overview of
>>>>>> this
>>>>>>>>>>>>>>>>>>>> issue,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> generally a surge of output events which affect
>>>>>> performance
>>>>>>>>>>>>>>>>>>>> moreso
>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> flush or prefixScan itself.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thoughts on any of these are greatly appreciated,
>>>> since
>>>>>>> these
>>>>>>>>>>>>>>>>>>>> elements
>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> really the cornerstone of the whole design. I can put
>>>> up
>>>>>>>>>>>>>>>>>>>> the code
>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>> written against 1.0.2 if we so desire, but first I was
>>>>>>>>>>>>>>>>>>>> hoping to
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>> tackle some of the fundamental design proposals.
>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>> Adam
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Mon, Jul 23, 2018 at 10:05 AM, Adam Bellemare <
>>>>>>>>>>>>>>>>>>>> adam.bellemare@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Here is the new discussion thread for KIP-213. I
>>>> picked
>>>>>>>>>>>>>>>>>>>> back up on
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> as this is something that we too at Flipp are now
>>>>> running
>>>>>> in
>>>>>>>>>>>>>>>>>>>>> production.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Jan started this last year, and I know that Trivago is
>>>>>> also
>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> similar in production, at least in terms of APIs and
>>>>>>>>>>>>>>>>>>>> functionality.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>> confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>> 213+Support+non-key+joining+in+KTable
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I do have an implementation of the code for Kafka
>>>> 1.0.2
>>>>>>> (our
>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>> production version) but I won't post it yet as I
>>>> would
>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> focus
>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>>>>>>> workflow and design first. That being said, I also
>>>> need
>>>>> to
>>>>>>> add
>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> clearer
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> integration tests (I did a lot of testing using a
>>>>>> non-Kafka
>>>>>>>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> framework) and clean up the code a bit more before
>>>>>> putting
>>>>>>>>>>>>>>>>>>>>> it in
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> PR
>>>>>>>>>>>>>>>>>>>>> against trunk (I can do so later this week likely).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Please take a look,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Adam Bellemare
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>
>