gharris1727 commented on code in PR #15601: URL: https://github.com/apache/kafka/pull/15601#discussion_r1554016667
########## streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSide.java: ########## @@ -33,28 +34,36 @@ public class TimestampedKeyAndJoinSide<K> { private final K key; private final long timestamp; - private final boolean leftSide; + private final JoinSide joinSide; - private TimestampedKeyAndJoinSide(final boolean leftSide, final K key, final long timestamp) { + private TimestampedKeyAndJoinSide(final JoinSide joinSide, final K key, final long timestamp) { this.key = Objects.requireNonNull(key, "key cannot be null"); - this.leftSide = leftSide; + this.joinSide = joinSide; this.timestamp = timestamp; } /** - * Create a new {@link TimestampedKeyAndJoinSide} instance if the provide {@code key} is not {@code null}. + * Create a new {@link TimestampedKeyAndJoinSide} instance if the provided {@code key} is not {@code null}. * - * @param leftSide True if the key is part of the left join side; False if it is from the right join side + * @param joinSide Whether the key is part of the {@link JoinSide#LEFT} side; or it is from the {@link JoinSide#RIGHT} side * @param key the key * @param <K> the type of the key - * @return a new {@link TimestampedKeyAndJoinSide} instance if the provide {@code key} is not {@code null} + * @return a new {@link TimestampedKeyAndJoinSide} instance if the provided {@code key} is not {@code null} */ - public static <K> TimestampedKeyAndJoinSide<K> make(final boolean leftSide, final K key, final long timestamp) { - return new TimestampedKeyAndJoinSide<>(leftSide, key, timestamp); + public static <K> TimestampedKeyAndJoinSide<K> make(final JoinSide joinSide, final K key, final long timestamp) { Review Comment: Since this is only used in tests now, I think you can remove this and replace the test call-sites with the new functions. Make sure to copy the javadoc to the new signatures too. ########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoin.java: ########## @@ -0,0 +1,201 @@ +/* + * 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 java.util.Optional; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.ValueJoinerWithKey; +import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class KStreamKStreamLeftJoin<K, V1, V2, VOut> extends KStreamKStreamJoin<K, V1, V2, VOut> { + private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); + + KStreamKStreamLeftJoin(final String otherWindowName, + final JoinWindowsInternal windows, + final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner, + final boolean outer, + final Optional<String> outerJoinWindowName, + final TimeTrackerSupplier sharedTimeTrackerSupplier) { + super(otherWindowName, windows, windows.beforeMs, windows.afterMs, joiner, outerJoinWindowName, + sharedTimeTrackerSupplier, outer); + } + + @Override + public Processor<K, V1, K, VOut> get() { + return new KStreamKStreamLeftJoinProcessor(); + } + + private class KStreamKStreamLeftJoinProcessor extends KStreamKStreamJoinProcessor { + @Override + public void process(final Record<K, V1> leftRecord) { + final long inputRecordTimestamp = leftRecord.timestamp(); + final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); + final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); + sharedTimeTracker.advanceStreamTime(inputRecordTimestamp); + + if (outer && leftRecord.key() == null && leftRecord.value() != null) { + final VOut leftJoinValue = joiner.apply(leftRecord.key(), leftRecord.value(), null); + context().forward(leftRecord.withValue(leftJoinValue)); + return; + } else if (StreamStreamJoinUtil.skipRecord(leftRecord, LOG, droppedRecordsSensor, context())) { + return; + } + + // Emit all non-joined records which window has closed + if (inputRecordTimestamp == sharedTimeTracker.streamTime) { + leftOuterJoinStore.ifPresent(store -> emitNonJoinedOuterRecords(store, leftRecord)); + } + + boolean needOuterJoin = outer; + try (final WindowStoreIterator<V2> iter = otherWindowStore.fetch(leftRecord.key(), timeFrom, timeTo)) { + while (iter.hasNext()) { + needOuterJoin = false; + final KeyValue<Long, V2> rightRecord = iter.next(); + final long rightRecordTimestamp = rightRecord.key; + + leftOuterJoinStore.ifPresent(store -> { + // use putIfAbsent to first read and see if there's any values for the key, + // if yes delete the key, otherwise do not issue a put; + // we may delete some values with the same key early but since we are going + // range over all values of the same key even after failure, since the other window-store + // is only cleaned up by stream time, so this is okay for at-least-once. + final TimestampedKeyAndJoinSide<K> leftKeyWithRightRecordTimestamp = + TimestampedKeyAndJoinSide.makeRightSide(leftRecord.key(), rightRecordTimestamp); + store.putIfAbsent(leftKeyWithRightRecordTimestamp, null); + }); + + final VOut joinResult = joiner.apply(leftRecord.key(), leftRecord.value(), rightRecord.value); + context().forward(leftRecord.withValue(joinResult) + .withTimestamp(Math.max(inputRecordTimestamp, rightRecordTimestamp))); + } + + if (needOuterJoin) { + // The maxStreamTime contains the max time observed in both sides of the join. + // Having access to the time observed in the other join side fixes the following + // problem: + // + // Say we have a window size of 5 seconds + // 1. A non-joined record with time T10 is seen in the left-topic (maxLeftStreamTime: 10) + // The record is not processed yet, and is added to the outer-join store + // 2. A non-joined record with time T2 is seen in the right-topic (maxRightStreamTime: 2) + // The record is not processed yet, and is added to the outer-join store + // 3. A joined record with time T11 is seen in the left-topic (maxLeftStreamTime: 11) + // It is time to look at the expired records. T10 and T2 should be emitted, but + // because T2 was late, then it is not fetched by the window store, so it is not processed + // + // See KStreamKStreamLeftJoinTest.testLowerWindowBound() tests + // + // This condition below allows us to process the out-of-order records without the need + // to hold it in the temporary outer store + if (!leftOuterJoinStore.isPresent() || timeTo < sharedTimeTracker.streamTime) { + final VOut leftJoinValue = joiner.apply(leftRecord.key(), leftRecord.value(), null); + context().forward(leftRecord.withValue(leftJoinValue)); + } else { + sharedTimeTracker.updatedMinTime(inputRecordTimestamp); + leftOuterJoinStore.ifPresent(store -> { + final TimestampedKeyAndJoinSide<K> leftKey = + TimestampedKeyAndJoinSide.makeLeftSide(leftRecord.key(), inputRecordTimestamp); + final LeftOrRightValue<V1, V2> leftValue = LeftOrRightValue.makeLeftValue(leftRecord.value()); + store.put(leftKey, leftValue); + }); + } + } + } + } + + private void emitNonJoinedOuterRecords( + final KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> store, + final Record<K, V1> record) { + + // calling `store.all()` creates an iterator what is an expensive operation on RocksDB; + // to reduce runtime cost, we try to avoid paying those cost + + // only try to emit left/outer join results if there _might_ be any result records + if (sharedTimeTracker.minTime + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) { + return; + } + // throttle the emit frequency to a (configurable) interval; + // we use processing time to decouple from data properties, + // as throttling is a non-functional performance optimization + if (internalProcessorContext.currentSystemTimeMs() < sharedTimeTracker.nextTimeToEmit) { + return; + } + + // Ensure `nextTimeToEmit` is synced with `currentSystemTimeMs`, if we dont set it everytime, + // they can get out of sync during a clock drift + sharedTimeTracker.nextTimeToEmit = internalProcessorContext.currentSystemTimeMs(); + sharedTimeTracker.advanceNextTimeToEmit(); + + // reset to MAX_VALUE in case the store is empty + sharedTimeTracker.minTime = Long.MAX_VALUE; + + try (final KeyValueIterator<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> it = store.all()) { + TimestampedKeyAndJoinSide<K> prevKey = null; + + while (it.hasNext()) { + final KeyValue<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> next = it.next(); + final TimestampedKeyAndJoinSide<K> timestampedKeyAndJoinSide = next.key; + final LeftOrRightValue<V1, V2> value = next.value; + final K key = timestampedKeyAndJoinSide.getKey(); + final long timestamp = timestampedKeyAndJoinSide.getTimestamp(); + sharedTimeTracker.minTime = timestamp; + + // Skip next records if window has not closed + final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide); + if (sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs + >= sharedTimeTracker.streamTime) { + continue; // there are possibly candidates left on the other outerJoin-side + } + + final VOut nullJoinedValue = joiner.apply(key, value.getLeftValue(), value.getRightValue()); Review Comment: Pull this out into getNullJoinedValue like the other side. ########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoin.java: ########## @@ -0,0 +1,201 @@ +/* + * 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 java.util.Optional; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.ValueJoinerWithKey; +import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class KStreamKStreamLeftJoin<K, V1, V2, VOut> extends KStreamKStreamJoin<K, V1, V2, VOut> { + private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); + + KStreamKStreamLeftJoin(final String otherWindowName, + final JoinWindowsInternal windows, + final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner, + final boolean outer, + final Optional<String> outerJoinWindowName, + final TimeTrackerSupplier sharedTimeTrackerSupplier) { + super(otherWindowName, windows, windows.beforeMs, windows.afterMs, joiner, outerJoinWindowName, Review Comment: I like this constructor refactor for the windows.beforeMs and the windows.afterMs! :+1+ ########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java: ########## @@ -16,276 +16,98 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; +import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; + +import java.util.Optional; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker; import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; 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.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.LeftOrRightValue; import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Optional; - -import static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; -import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; - -class KStreamKStreamJoin<K, V1, V2, VOut> implements ProcessorSupplier<K, V1, K, VOut> { - private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); +abstract class KStreamKStreamJoin<K, V1, V2, VOut> implements ProcessorSupplier<K, V1, K, VOut> { + protected final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner; + protected final long joinGraceMs; + protected final boolean outer; + protected final long joinBeforeMs; + protected final long joinAfterMs; private final String otherWindowName; - private final long joinBeforeMs; - private final long joinAfterMs; - private final long joinGraceMs; private final boolean enableSpuriousResultFix; private final long windowsBeforeMs; private final long windowsAfterMs; - - private final boolean outer; - private final boolean isLeftSide; private final Optional<String> outerJoinWindowName; - private final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner; private final TimeTrackerSupplier sharedTimeTrackerSupplier; - KStreamKStreamJoin(final boolean isLeftSide, - final String otherWindowName, - final JoinWindowsInternal windows, - final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner, - final boolean outer, - final Optional<String> outerJoinWindowName, - final TimeTrackerSupplier sharedTimeTrackerSupplier) { - this.isLeftSide = isLeftSide; + KStreamKStreamJoin(final String otherWindowName, + final JoinWindowsInternal windows, + final long joinBeforeMs, + final long joinAfterMs, + final ValueJoinerWithKey<? super K, ? super V1, ? super V2, ? extends VOut> joiner, + final Optional<String> outerJoinWindowName, + final TimeTrackerSupplier sharedTimeTrackerSupplier, + final boolean outer) { this.otherWindowName = otherWindowName; - if (isLeftSide) { - this.joinBeforeMs = windows.beforeMs; - this.joinAfterMs = windows.afterMs; - } else { - this.joinBeforeMs = windows.afterMs; - this.joinAfterMs = windows.beforeMs; - } this.windowsAfterMs = windows.afterMs; this.windowsBeforeMs = windows.beforeMs; - this.joinGraceMs = windows.gracePeriodMs(); + this.joinBeforeMs = joinBeforeMs; + this.joinAfterMs = joinAfterMs; this.enableSpuriousResultFix = windows.spuriousResultFixEnabled(); this.joiner = joiner; - this.outer = outer; + this.joinGraceMs = windows.gracePeriodMs(); this.outerJoinWindowName = outerJoinWindowName; this.sharedTimeTrackerSupplier = sharedTimeTrackerSupplier; + this.outer = outer; } - @Override - public Processor<K, V1, K, VOut> get() { - return new KStreamKStreamJoinProcessor(); - } - - private class KStreamKStreamJoinProcessor extends ContextualProcessor<K, V1, K, VOut> { - private WindowStore<K, V2> otherWindowStore; - private Sensor droppedRecordsSensor; - private Optional<KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>> outerJoinStore = Optional.empty(); - private InternalProcessorContext<K, VOut> internalProcessorContext; - private TimeTracker sharedTimeTracker; + protected abstract class KStreamKStreamJoinProcessor extends ContextualProcessor<K, V1, K, VOut> { + protected WindowStore<K, V2> otherWindowStore; + protected Sensor droppedRecordsSensor; + protected Optional<KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>> leftOuterJoinStore = + Optional.empty(); + protected Optional<KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V2, V1>>> rightOuterJoinStore = + Optional.empty(); Review Comment: The left store is only used by the Left class, and the right store is only used by the Right class. These can be combined into a single outerJoinStore, and the KStreamKStreamJoinProcessor can take generic arguments <Va, Vb> that determine the types in the joinStore signature. The LeftJoinProcessor can then set Va=V1 and Vb=V2, while the RightJoinProcessor sets Va=V2 and Vb=V1. -- 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