guozhangwang commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r980366498


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java:
##########
@@ -375,6 +382,48 @@ private void mergeDuplicateSourceNodes() {
         }
     }
 
+
+    /**
+     * The self-join rewriting can be applied if the StreamStreamJoinNode has 
a single parent.
+     * If the join is a self-join, remove the node KStreamJoinWindow 
corresponding to the
+     * right argument of the join (the "other"). The join node may have 
multiple siblings but for
+     * this rewriting we only care about the ThisKStreamJoinWindow and the 
OtherKStreamJoinWindow.
+     * We iterate over all the siblings to identify these two nodes so that we 
can remove the
+     * latter.
+     */
+    @SuppressWarnings("unchecked")
+    private void rewriteSingleStoreSelfJoin(
+        final GraphNode currentNode, final Map<GraphNode, Boolean> visited) {
+        visited.put(currentNode, true);
+        if (currentNode instanceof StreamStreamJoinNode && 
currentNode.parentNodes().size() == 1) {
+            final StreamStreamJoinNode joinNode = (StreamStreamJoinNode) 
currentNode;
+            // Remove JoinOtherWindowed node
+            final GraphNode parent = 
joinNode.parentNodes().stream().findFirst().get();
+            GraphNode left = null, right = null;
+            for (final GraphNode child: parent.children()) {
+                if (child instanceof WindowedStreamProcessorNode && 
child.buildPriority() < joinNode.buildPriority()) {
+                    if 
(child.nodeName().equals(joinNode.getThisWindowedStreamProcessorParameters().processorName()))
 {
+                        left = child;
+                    } else if 
(child.nodeName().equals(joinNode.getOtherWindowedStreamProcessorParameters().processorName()))
 {
+                        right = child;
+                    }
+                }
+            }
+            // Sanity check
+            if (left != null && right != null && left.buildPriority() < 
right.buildPriority()) {
+                parent.removeChild(right);
+                joinNode.setSelfJoin();
+            } else {
+                throw new StreamsException(String.format("Expected the left 
node %s to have smaller build priority than the right node %s.", left, right));

Review Comment:
   nit: I'd suggest we throw an unchecked illegal-state-exception directly 
since this should never happen, while streams exception is a checked exception.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java:
##########
@@ -92,17 +96,38 @@ public void writeToTopology(final InternalTopologyBuilder 
topologyBuilder) {
         final String thisWindowedStreamProcessorName = 
thisWindowedStreamProcessorParameters.processorName();
         final String otherWindowedStreamProcessorName = 
otherWindowedStreamProcessorParameters.processorName();
 
-        topologyBuilder.addProcessor(thisProcessorName, 
thisProcessorParameters().processorSupplier(), thisWindowedStreamProcessorName);
-        topologyBuilder.addProcessor(otherProcessorName, 
otherProcessorParameters().processorSupplier(), 
otherWindowedStreamProcessorName);
-        
topologyBuilder.addProcessor(mergeProcessorParameters().processorName(), 
mergeProcessorParameters().processorSupplier(), thisProcessorName, 
otherProcessorName);
-        topologyBuilder.addStateStore(thisWindowStoreBuilder, 
thisWindowedStreamProcessorName, otherProcessorName);
-        topologyBuilder.addStateStore(otherWindowStoreBuilder, 
otherWindowedStreamProcessorName, thisProcessorName);
-
-        if (enableSpuriousResultFix) {
-            outerJoinWindowStoreBuilder.ifPresent(builder -> 
topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName));
+        if (isSelfJoin) {
+            
topologyBuilder.addProcessor(selfJoinProcessorParameters.processorName(), 
selfJoinProcessorParameters.processorSupplier(), 
thisWindowedStreamProcessorName);
+            topologyBuilder.addStateStore(thisWindowStoreBuilder, 
thisWindowedStreamProcessorName, selfJoinProcessorParameters.processorName());
+        } else {
+            topologyBuilder.addProcessor(thisProcessorName, 
thisProcessorParameters().processorSupplier(), thisWindowedStreamProcessorName);
+            topologyBuilder.addProcessor(otherProcessorName, 
otherProcessorParameters().processorSupplier(), 
otherWindowedStreamProcessorName);
+            
topologyBuilder.addProcessor(mergeProcessorParameters().processorName(), 
mergeProcessorParameters().processorSupplier(), thisProcessorName, 
otherProcessorName);
+            topologyBuilder.addStateStore(thisWindowStoreBuilder, 
thisWindowedStreamProcessorName, otherProcessorName);
+            topologyBuilder.addStateStore(otherWindowStoreBuilder, 
otherWindowedStreamProcessorName, thisProcessorName);
+
+            if (enableSpuriousResultFix) {
+                outerJoinWindowStoreBuilder.ifPresent(builder -> 
topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName));
+            }
         }
     }
 
+    public void setSelfJoin() {
+        this.isSelfJoin = true;
+    }
+
+    public boolean getSelfJoin() {

Review Comment:
   Seems not used?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamSelfJoin.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import static 
org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.ValueJoinerWithKey;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.ProcessorSupplier;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class KStreamKStreamSelfJoin<K, V1, V2, VOut> implements ProcessorSupplier<K, 
V1, K, VOut> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KStreamKStreamSelfJoin.class);
+
+    private final String windowName;
+    private final long joinThisBeforeMs;
+    private final long joinThisAfterMs;
+    private final long joinOtherBeforeMs;
+    private final long joinOtherAfterMs;
+    private final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? 
extends VOut> joinerThis;
+
+    private final TimeTracker sharedTimeTracker;
+
+    KStreamKStreamSelfJoin(
+        final String windowName,
+        final JoinWindowsInternal windows,
+        final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends 
VOut> joinerThis,
+        final TimeTracker sharedTimeTracker) {
+
+        this.windowName = windowName;
+        this.joinThisBeforeMs = windows.beforeMs;
+        this.joinThisAfterMs = windows.afterMs;
+        this.joinOtherBeforeMs = windows.afterMs;
+        this.joinOtherAfterMs = windows.beforeMs;
+        this.joinerThis = joinerThis;
+        this.sharedTimeTracker = sharedTimeTracker;
+    }
+
+    @Override
+    public Processor<K, V1, K, VOut> get() {
+        return new KStreamKStreamSelfJoinProcessor();
+    }
+
+    private class KStreamKStreamSelfJoinProcessor extends 
ContextualProcessor<K, V1, K, VOut> {
+        private WindowStore<K, V2> windowStore;
+        private Sensor droppedRecordsSensor;
+
+        @Override
+        public void init(final ProcessorContext<K, VOut> context) {
+            super.init(context);
+
+            final StreamsMetricsImpl metrics = (StreamsMetricsImpl) 
context.metrics();
+            droppedRecordsSensor = 
droppedRecordsSensor(Thread.currentThread().getName(), 
context.taskId().toString(), metrics);
+            windowStore = context.getStateStore(windowName);
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void process(final Record<K, V1> record) {
+            if (StreamStreamJoinUtil.skipRecord(record, LOG, 
droppedRecordsSensor, context())) {
+                return;
+            }
+
+            final long inputRecordTimestamp = record.timestamp();
+            long timeFrom = Math.max(0L, inputRecordTimestamp - 
joinThisBeforeMs);
+            long timeTo = Math.max(0L, inputRecordTimestamp + joinThisAfterMs);
+            boolean emittedJoinWithSelf = false;
+            final Record selfRecord = record
+                .withValue(joinerThis.apply(record.key(), record.value(), (V2) 
record.value()))
+                .withTimestamp(inputRecordTimestamp);
+            sharedTimeTracker.advanceStreamTime(inputRecordTimestamp);
+
+            // Join current record with other
+            try (final WindowStoreIterator<V2> iter = windowStore.fetch(

Review Comment:
   nit: put the fetch params along with `fetch` in a single line?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java:
##########
@@ -198,7 +207,8 @@ public <K, V1, V2, VOut> KStream<K, VOut> join(final 
KStream<K, V1> lhs,
                    
.withOtherWindowedStreamProcessorParameters(otherWindowStreamProcessorParams)
                    .withOuterJoinWindowStoreBuilder(outerJoinWindowStore)
                    .withValueJoiner(joiner)
-                   .withNodeName(joinMergeName);
+                   .withNodeName(joinMergeName)
+                   .withSelfJoinProcessorParameters(selfJoinProcessorParams);

Review Comment:
   Not a strong opinion, since I know we are messing the logical planner with 
physical info quite badly already.. but I'm wondering if we could defer this in 
`StreamStreamJoinNode#writeToTopology` inside the `isSelfJoin` condition? We 
still have all the pieces we need: 1) left store name, 2) both join window 
specs. 3) joiner, 4) time tracker, from other params, so that in 
`writeToTopology` we can still generate the `KStreamKStreamSelfJoin` object if 
self-join is enabled.
   
   My motivation is just to not spill more physical node info into logical 
planning phase.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/WindowedStreamProcessorNode.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals.graph;
+
+public class WindowedStreamProcessorNode<K, V> extends ProcessorGraphNode<K, 
V> {
+
+    private final String windowStoreName;
+    private final String processorName;
+
+    /**
+     * Create a node representing a Stream Join Window processor.
+     */
+    public WindowedStreamProcessorNode(final String windowStoreName,
+                                       final String processorName,

Review Comment:
   Do we need the processorName passed in here? I thought that should always be 
== processorParameters.processorName()?



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to