[ 
https://issues.apache.org/jira/browse/KAFKA-7497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646892#comment-16646892
 ] 

Matthias J. Sax commented on KAFKA-7497:
----------------------------------------

It is correct, that a topic cannot be consumed twice (cf. 
https://issues.apache.org/jira/browse/KAFKA-6687) and I also agree that a 
self-join operator would be useful.

Once could express a self-joining like this:
{noformat}
KStream stream = builder.stream(...);
stream.join(stream, ...);{noformat}
However, the execution of the join would not be efficient, as two state stores 
with two changelog topics would be created (both containing the exact same 
data). Also, and this seems to be the most severs issue, each record would join 
with itself, what is actually not desired...

I marked this as "needs-kip" but I am not 100% sure if we would need a KIP 
though. Maybe, Kafka Streams could detect internally, that left hand side 
KStream and right hand side KStream is the same object and just use a different 
operator implementation (ie, a dedicated self-join processor). This way, no 
public API change would be required.

> Kafka Streams should support self-join on streams
> -------------------------------------------------
>
>                 Key: KAFKA-7497
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7497
>             Project: Kafka
>          Issue Type: New Feature
>          Components: streams
>            Reporter: Robin Moffatt
>            Priority: Major
>              Labels: needs-kip
>
> There are valid reasons to want to join a stream to itself, but Kafka Streams 
> does not currently support this ({{Invalid topology: Topic foo has already 
> been registered by another source.}}).  To perform the join requires creating 
> a second stream as a clone of the first, and then doing a join between the 
> two. This is a clunky workaround and results in unnecessary duplication of 
> data.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to