Hi Matthias,

Hi Matthias,



Thanks for your comments.



I would like to clarify the use case a little more to show why existing
table-table foreign key join will not work for the use case I am trying to
address.

Let’s consider the very simple use case with the parent messages in one
Kafka topic (‘order event’ messages that also contain some key order info)
and the child messages in another topic (‘order items’ messages with an
additional info for the order). Relationship between parent and child
messages is 1:1. Also ‘order items’ message has OrderID as one of its
fields (foreign key).



The requirement is to combine info of the parent ‘order event’ message with
child ‘order items’ message using foreign key and to send it only once to
the target system as one ‘complete order’ message for each new ‘order
event’ message.

Please note that the new messages which are related to order items (create,
update, delete) should not trigger the resulting ‘complete order’ message).



>From the above requirements we can state the following:

1.     Order events are unique and never updated or deleted; they can only
be replayed if we need to recover the event stream. For our order example I
would use OrderID as an event key but if we use the KTable to represent
events then events with the same OrderID will overwrite each other. This
may or may not cause some issues but using the stream to model seems to be
a more correct approach from at least performance point of view.

2.     We do not want updates from the “order items” table on the right
side of the join to generate an output since only events should be the
trigger for output messages in our scenario. This is aligned with the
stream-table join behavior rather than table-table join when updates are
coming from both sides

3.     Stream-table join will give us resulting stream which is more align
with our output requirements than the table that would be result of
table-table join



Requirement #2 above is the most important one and it can not be achieved
with existing table-table join on foreign key.



I also stated that the foreign key table in table-table join is on the
‘wrong’ side for our order management use case. By this I just meant that
in stream-table join I am proposing the foreign key table needs to be on
the right side and on the existing table-table join it is on the left. This
is however is irrelevant since we can not use table-table join anyway for
the reason #2 above.



You made a good point about aggregation of child messages for a more
complex use case of 1:n relation between parent and children. Initially I
was thinking that aggregation will be just a separate operation that could
be added after we performed a foreign key join. Now I realize that it will
not be possible to do it after.

Maybe there could be a flag to stream-table foreign key join that would
indicate if we want this join to aggregate children or not?



What do you think?

Regards,



Igor


On Fri, Aug 4, 2023 at 10:01 PM Matthias J. Sax <mj...@apache.org> wrote:

> Thanks a lot for providing more background. It's getting much clear to
> me now.
>
> Couple of follow up questions:
>
>
> > It is not possible to use table-table join in this case because
> triggering
> > events are supplied separately from the actual data entity that needs to
> be
> > "assembled" and these events could only be presented as KStream due to
> > their nature.
>
> Not sure if I understand this part? Why can't those events not
> represented as a KTable. You say "could only be presented as KStream due
> to their nature" -- what do you mean by this?
>
> In the end, my understanding is the following (using the example for the
> KIP):
>
> For the shipments <-> orders and order-details <-> orders join, shipment
> and order-details are the fact table, what is "reverse" to what you
> want? Using existing FK join, it would mean you get two enriched tables,
> that you cannot join to each other any further (because we don't support
> n:m join): in the end, shipmentId+orderDetailId would be the PK of such
> a n:m join?
>
> If that's correct, (just for the purpose to make sure I understand
> correctly), if we would add an n:m join, you could join shipment <->
> order-details first, and use a FK join to enrich the result with orders.
> -- In addition, you could also do a FK join to event if you represent
> events as a table (this relates to my question from above, why events
> cannot be represented as a KTable).
>
>
> A the KIP itself, I am still wondering about details: if we get an event
> in, and we do a lookup into the "FK table" and find multiple matches,
> would we emit multiple results? This would kinda defeat the purpose to
> re-assemble everything into a single entity? (And it might require an
> additional aggregation downstream to put the entity together.) -- Or
> would we join the singe event, with all found table rows, and emit a
> single "enriched" event?
>
>
> Thus, I am actually wondering, if you would not pre-process both
> shipment and order-details table, via `groupBy(orderId)` and assemble a
> list (or similar) of alls shipments (or order-details) per order? If you
> do this pre-processing, you can do a PK-PK (1:1) join with the orders
> table, and also do a stream-table join to enrich your events will the
> full order information?
>
>
>
> -Matthias
>
> On 7/26/23 7:13 AM, Igor Fomenko wrote:
> > Hello Matthias,
> >
> > Thank you for this response. It provides the context for a good
> discussion
> > related to the need for this new interface.
> >
> > The use case I have in mind is not really a stream enrichment which
> usually
> > implies that the event has a primary key to some external info and that
> > external info could be just looked up in some other data source.
> >
> > The pattern this KIP proposes is more akin to the data entity assembly
> > pattern from the persistence layer so it is not purely integration
> pattern
> > but rather a pattern that enables an event stream from persistence layer
> of
> > a data source application. The main driver here is the ability to stream
> a
> > data entity of any complexity (complexity in terms of the relational
> model)
> > from an application database to some data consumers. The technical
> > precondition here is of course that data is already extracted from the
> > relational database with something like Change Data Capture (CDC) and
> > placed to Kafka topics. Also due to CDC limitations, each database table
> > that is related to the entity relational data model is extracted to the
> > separate Kafka topic.
> >
> > So to answer you first question the entity that needs to be "assembled"
> > from Kafka topics in the very common use case has 1:n relations where 1
> > corresponds to the triggering event enriched with the data from the main
> > (or parent) table of the data entity (for example completion of the
> > purchase order event + order data from the order table) and n corresponds
> > to the many children that needs to be joined with the order table to have
> > the full data entity (for example multiple line items of the purchase
> order
> > needs to be added from the line items child table).
> >
> > It is not possible to use table-table join in this case because
> triggering
> > events are supplied separately from the actual data entity that needs to
> be
> > "assembled" and these events could only be presented as KStream due to
> > their nature. Also currently the FK table in table-table join is on the
> > "wrong" side of the join.
> > It is possible to use existing stream-table join only to get data from
> the
> > parent entity table (order table) because the event to order is 1:1.
> After
> > that it is required to add "children" tables of the order to complete
> > entity assembly, these childered are related as 1:n with foreign key
> fields
> > in each child table (which is order ID).
> >
> > This use case is typically implemented with some sort of ESB (like
> > Mulesoft) where ESB receives an event and then uses JDBC adapter to issue
> > SQL query with left join on foreign key for child tables. ESB then loops
> > through the returned record set to assemble the full data entity. However
> > in many cases for various architecture reasons there is a desire to
> remove
> > JDBC queries from the data source and replace it with CDC streaming data
> to
> > Kafka. So in that case assembling data entities from Kafka topics instead
> > of JDBC would be beneficial.
> >
> > Please let me know what you think.
> >
> > Regards,
> >
> > Igor
> >
> > On Tue, Jul 25, 2023 at 5:53 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >> Igor,
> >>
> >> thanks for the KIP. Interesting proposal. I am wondering a little bit
> >> about the use-case and semantics, and if it's really required to add
> >> what you propose? Please correct me if I am wrong.
> >>
> >> In the end, a stream-table join is a "stream enrichment" (via a table
> >> lookup). Thus, it's inherently a 1:1 join (in contrast to a FK
> >> table-table join which is a n:1 join).
> >>
> >> If this assumption is correct, and you have data for which the table
> >> side join attribute is in the value, you could actually repartition the
> >> table data using the join attribute as the PK of the table.
> >>
> >> If my assumption is incorrect, and you say you want to have a 1:n join
> >> (note that I intentionally reversed from n:1 to 1:n), I would rather
> >> object, because it seems to violate the idea to "enrich" a stream, what
> >> means that each input record produced an output record, not multiple?
> >>
> >> Also note: for a FK table-table join, we use the forgeinKeyExtractor to
> >> get the join attribute from the left input table (which corresponds to
> >> the KStream in your KIP; ie, it's a n:1 join), while you propose to use
> >> the foreignKeyExtractor to be applied to the KTable (which is the right
> >> input, and thus it would be a 1:n join).
> >>
> >> Maybe you can clarify the use case a little bit. For the current KIP
> >> description I only see the 1:1 join case, what would mean we might not
> >> need such a feature?
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 7/24/23 11:36 AM, Igor Fomenko wrote:
> >>> Hello developers of the Kafka Streams,
> >>>
> >>> I would like to start discussion on KIP-955: Add stream-table join on
> >>> foreign key
> >>> <
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-955%3A+Add+stream-table+join+on+foreign+key
> >>>
> >>> This KIP proposes the new API to join KStrem with KTable based on
> foreign
> >>> key relation.
> >>> Ths KIP was inspired by one of my former projects to integrate RDBMS
> >>> databases with data consumers using Change Data Capture and Kafka.
> >>> If we had the capability in Kafka Stream to join KStream with KTable on
> >>> foreign key this would simplify our implementation significantly.
> >>>
> >>> Looking forward to your feedback and discussion.
> >>>
> >>> Regards,
> >>>
> >>> Igor
> >>>
> >>
> >
>

Reply via email to