You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/09/15 08:48:25 UTC

[GitHub] [kafka] vpapavas opened a new pull request, #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

vpapavas opened a new pull request, #12644:
URL: https://github.com/apache/kafka/pull/12644

   KIP can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-862%3A+Self-join
   
   It only applies to Stream-Stream joins and not n-way self-joins. 
   
   This is an inner-join topology (without the optimization)
   ```
   Topologies:
      Sub-topology: 0
       Source: KSTREAM-SOURCE-0000000000 (topics: [topic2])
         --> KSTREAM-WINDOWED-0000000001, KSTREAM-WINDOWED-0000000002
       Processor: KSTREAM-WINDOWED-0000000001 (stores: [KSTREAM-JOINTHIS-0000000003-store])
         --> KSTREAM-JOINTHIS-0000000003
         <-- KSTREAM-SOURCE-0000000000
       Processor: KSTREAM-WINDOWED-0000000002 (stores: [KSTREAM-JOINOTHER-0000000004-store])
         --> KSTREAM-JOINOTHER-0000000004
         <-- KSTREAM-SOURCE-0000000000
       Processor: KSTREAM-JOINOTHER-0000000004 (stores: [KSTREAM-JOINTHIS-0000000003-store])
         --> KSTREAM-MERGE-0000000005
         <-- KSTREAM-WINDOWED-0000000002
       Processor: KSTREAM-JOINTHIS-0000000003 (stores: [KSTREAM-JOINOTHER-0000000004-store])
         --> KSTREAM-MERGE-0000000005
         <-- KSTREAM-WINDOWED-0000000001
       Processor: KSTREAM-MERGE-0000000005 (stores: [])
         --> KSTREAM-PROCESSOR-0000000006
         <-- KSTREAM-JOINTHIS-0000000003, KSTREAM-JOINOTHER-0000000004
       Processor: KSTREAM-PROCESSOR-0000000006 (stores: [])
         --> none
         <-- KSTREAM-MERGE-0000000005
   ```
   ![inner-join](https://user-images.githubusercontent.com/7738368/186751759-92899e7f-0bfa-4266-a37d-2c2f70f4dda7.png)
   
   
   and this is the optimized self-join topology
   
   ```
   Topologies:
      Sub-topology: 0
       Source: KSTREAM-SOURCE-0000000000 (topics: [topic1])
         --> KSTREAM-WINDOWED-0000000001
       Processor: KSTREAM-WINDOWED-0000000001 (stores: [KSTREAM-JOINTHIS-0000000003-store])
         --> KSTREAM-MERGE-0000000005
         <-- KSTREAM-SOURCE-0000000000
       Processor: KSTREAM-MERGE-0000000005 (stores: [KSTREAM-JOINTHIS-0000000003-store])
         --> KSTREAM-PROCESSOR-0000000006
         <-- KSTREAM-WINDOWED-0000000001
       Processor: KSTREAM-PROCESSOR-0000000006 (stores: [])
         --> none
         <-- KSTREAM-MERGE-0000000005
   ```
   ![self-join](https://user-images.githubusercontent.com/7738368/186751945-5f92369c-eea6-4ed1-9c41-7cfaa30ba117.png)
   -----
   Testing: Unit tests (Integration and upgrade test in follow up PR)
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r977472123


##########
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(
+                record.key(), timeFrom, timeTo)) {

Review Comment:
   We need to loop over the records twice to ensure that the results match the output of the inner-join. The first loop simulates the join triggered by the left. The second loop simulates the join triggered by the right. I could store the results in a data structure and iterate over it instead of doing a second fetch, but we don't know how many these records might be and I didn't want to cause memory issues. So, I decided to sacrifice CPU for memory.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r977470895


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java:
##########
@@ -356,12 +383,55 @@ private void mergeDuplicateSourceNodes() {
         }
     }
 
-    private void optimizeKTableSourceTopics() {
+
+    /**
+     * 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

Review Comment:
   The current JoinNode might have other JoinNodes as siblings. We need to differentiate between the `WindowedStreamProcessorNode` nodes that belong the current node versus others.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r981685201


##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamSelfJoinTest.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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 java.time.Duration.ofMillis;
+import static java.time.Duration.ofSeconds;
+
+import java.util.List;
+import java.util.Properties;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.JoinWindows;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.StreamJoined;
+import org.apache.kafka.streams.kstream.ValueJoiner;
+import org.apache.kafka.test.MockApiProcessor;
+import org.apache.kafka.test.MockApiProcessorSupplier;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.Test;
+
+public class KStreamKStreamSelfJoinTest {
+    private final String topic1 = "topic1";
+    private final String topic2 = "topic2";
+    private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+
+    @Test
+    public void shouldMatchInnerJoinWithSelfJoinWithSingleStream() {
+        props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, StreamsConfig.METRICS_LATEST);
+        props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
+        final ValueJoiner<String, String, String> valueJoiner = (v, v2) -> v + v2;
+        final List<KeyValueTimestamp<String, String>> expected;

Review Comment:
   Thanks for the thoughtful tests comparing the optimized topology with non-optimized topology, really like it!
   
   Just one thought: it seems we do not need to create the `StreamsBuilder` twice, but instead we can just create it once from the same topic (since they get the same data as well), and then call `build` twice each different `props` to generate two topologies with different client ids so that they will re-run from the beginning of the input topic for the test driver, is that doable?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] aheyne commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
aheyne commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r978742684


##########
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(
+                record.key(), timeFrom, timeTo)) {

Review Comment:
   Yes, definitely. Technically I think we're doing a Self Cartesian (Cross) Join. These two points should let us do that: 
   
   > - for self joins there’s usually a filter along with the join (like in SQL CREATE stream2 as SELECT .. FROM stream1 A, stream1 B WHERE A.field <> B.field, which would be translated as a stream.join().filter() topology) to filter on some value conditions because otherwise users are just squaring the stream events.
   > - even without the join conditions translated as a filter right after the join, users may not want to have two joined records joined(value1, value2) and joined(value2, value1) in their final streams, and they may want to do some de-duping to just keep one, and hence they’d have to add such de-duping operator after the join anyways.
   



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r981688919


##########
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:
   Ack.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r978508669


##########
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(
+                record.key(), timeFrom, timeTo)) {

Review Comment:
   Hi @aheyne ! You mean sacrificing either of the sides would work for your use case? I have opened the follow-up ticket to optimize the performance of the self-join if the ordering of the records is not important. Would that work for you? https://issues.apache.org/jira/browse/KAFKA-14251



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] aheyne commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
aheyne commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r978077841


##########
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(
+                record.key(), timeFrom, timeTo)) {

Review Comment:
   For my use case we always drop the right side of the join, so sacrificing neither would be nice. Would adding a config for this over complicate things? 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r981415910


##########
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:
   It's used in the tests



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on PR #12644:
URL: https://github.com/apache/kafka/pull/12644#issuecomment-1255115493

   I opened the follow-up ticket for improving runtime by doing a single-loop https://issues.apache.org/jira/browse/KAFKA-14251. 


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] vpapavas commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
vpapavas commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r977793194


##########
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(
+                record.key(), timeFrom, timeTo)) {
+                while (iter.hasNext()) {
+                    final KeyValue<Long, V2> otherRecord = iter.next();
+                    final long otherRecordTimestamp = otherRecord.key;
+
+                    // Join this with other
+                    context().forward(
+                        record.withValue(joinerThis.apply(
+                                record.key(), record.value(), otherRecord.value))
+                            .withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp)));
+                }
+            }
+
+            // Needs to be in a different loop to ensure correct ordering of records where
+            // correct ordering means it matches the output of an inner join.
+            timeFrom = Math.max(0L, inputRecordTimestamp - joinOtherBeforeMs);
+            timeTo = Math.max(0L, inputRecordTimestamp + joinOtherAfterMs);
+            try (final WindowStoreIterator<V2> iter2 = windowStore.fetch(
+                record.key(), timeFrom, timeTo)) {
+                while (iter2.hasNext()) {
+                    final KeyValue<Long, V2> otherRecord = iter2.next();
+                    final long otherRecordTimestamp = otherRecord.key;
+                    final long maxRecordTimestamp = Math.max(inputRecordTimestamp, otherRecordTimestamp);
+
+                    // This is needed so that output records follow timestamp order
+                    // Join this with self
+                    if (inputRecordTimestamp < maxRecordTimestamp && !emittedJoinWithSelf) {
+                        emittedJoinWithSelf = true;
+                        context().forward(selfRecord);
+                    }
+
+                    // Join other with current record
+                    context().forward(
+                        record
+                            .withValue(joinerThis.apply(record.key(), (V1) otherRecord.value, (V2) record.value()))
+                            .withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp)));
+                }
+            }
+
+            // Join this with self

Review Comment:
   @guozhangwang I tried doing the change in `KStreamJoinWindow` (inverting the lines)
   ```
    if (record.key() != null) {
                   context().forward(record);
                   // Every record basically starts a new window. We're using a window store mostly for the retention.
                   window.put(record.key(), record.value(), record.timestamp());
               }
   ```
   
   but it results in the self-record to get emitted multiple times*. This means that we still need logic to check if it was already emitted to emit only once. Which doesn't make the code any clearer so I decided against it. 
   
   * This happens because the record is already in the state store, so it joins with itself in the first loop but also in the second. 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12644: KAFKA-14209 : Rewrite self joins to use single state store 2/3

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on code in PR #12644:
URL: https://github.com/apache/kafka/pull/12644#discussion_r977863607


##########
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(
+                record.key(), timeFrom, timeTo)) {
+                while (iter.hasNext()) {
+                    final KeyValue<Long, V2> otherRecord = iter.next();
+                    final long otherRecordTimestamp = otherRecord.key;
+
+                    // Join this with other
+                    context().forward(
+                        record.withValue(joinerThis.apply(
+                                record.key(), record.value(), otherRecord.value))
+                            .withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp)));
+                }
+            }
+
+            // Needs to be in a different loop to ensure correct ordering of records where
+            // correct ordering means it matches the output of an inner join.
+            timeFrom = Math.max(0L, inputRecordTimestamp - joinOtherBeforeMs);
+            timeTo = Math.max(0L, inputRecordTimestamp + joinOtherAfterMs);
+            try (final WindowStoreIterator<V2> iter2 = windowStore.fetch(
+                record.key(), timeFrom, timeTo)) {
+                while (iter2.hasNext()) {
+                    final KeyValue<Long, V2> otherRecord = iter2.next();
+                    final long otherRecordTimestamp = otherRecord.key;
+                    final long maxRecordTimestamp = Math.max(inputRecordTimestamp, otherRecordTimestamp);
+
+                    // This is needed so that output records follow timestamp order
+                    // Join this with self
+                    if (inputRecordTimestamp < maxRecordTimestamp && !emittedJoinWithSelf) {
+                        emittedJoinWithSelf = true;
+                        context().forward(selfRecord);
+                    }
+
+                    // Join other with current record
+                    context().forward(
+                        record
+                            .withValue(joinerThis.apply(record.key(), (V1) otherRecord.value, (V2) record.value()))
+                            .withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp)));
+                }
+            }
+
+            // Join this with self

Review Comment:
   Got it, I missed that part.. thanks for checking and letting me know @vpapavas 



-- 
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-unsubscribe@kafka.apache.org

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