vpapavas commented on code in PR #12555:
URL: https://github.com/apache/kafka/pull/12555#discussion_r960845326
##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java:
##########
@@ -356,6 +381,95 @@ private void mergeDuplicateSourceNodes() {
}
}
+ /**
+ * If the join is a self-join, remove the node KStreamJoinWindow
corresponding to the
+ */
+ @SuppressWarnings("unchecked")
+ private void rewriteSelfJoin(final GraphNode currentNode, final
Set<GraphNode> visited) {
+ visited.add(currentNode);
+ if (currentNode instanceof StreamStreamJoinNode &&
isSelfJoin(currentNode)) {
+ ((StreamStreamJoinNode) currentNode).setSelfJoin();
+ // Remove JoinOtherWindowed node
+ final GraphNode parent =
currentNode.parentNodes().stream().findFirst().get();
+ GraphNode left = null, right = null;
+ for (final GraphNode child: parent.children()) {
+ if (child instanceof ProcessorGraphNode
+ && isStreamJoinWindowNode((ProcessorGraphNode) child)) {
+ if (left == null) {
+ left = child;
+ } else {
+ right = child;
+ }
+ }
+ }
+ // Sanity check
+ if (left != null && right != null && left.buildPriority() <
right.buildPriority()) {
+ parent.removeChild(right);
+ }
+ }
+ for (final GraphNode child: currentNode.children()) {
+ if (!visited.contains(child)) {
+ rewriteSelfJoin(child, visited);
Review Comment:
We can have n-way self-joins but they won't get optimized, only the first
one will get optimized. However, we can have topologies that have more than one
self-joins, not n-way but independent pairs. And all of them can and should be
optimized. I added a test case in
`InternalStreamsBuilderTest:shouldMarkAllStreamStreamJoinsAsSelfJoin`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]