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/22 20:54:59 UTC

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

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