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