cadonna commented on code in PR #15601:
URL: https://github.com/apache/kafka/pull/15601#discussion_r1604937635
##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java:
##########
@@ -187,17 +157,25 @@ public void process(final Record<K, V1> record) {
context().forward(record.withValue(joiner.apply(record.key(), record.value(),
null)));
} else {
sharedTimeTracker.updatedMinTime(inputRecordTimestamp);
- outerJoinStore.ifPresent(store -> store.put(
- TimestampedKeyAndJoinSide.make(isLeftSide,
record.key(), inputRecordTimestamp),
- LeftOrRightValue.make(isLeftSide,
record.value())));
+ putInOuterJoinStore(record);
}
}
}
}
+ protected abstract TimestampedKeyAndJoinSide<K> makeThisKey(final K
key, final long inputRecordTimestamp);
+
+ protected abstract LeftOrRightValue<VLeft, VRight> makeThisValue(final
VThis thisValue);
+
+ protected abstract TimestampedKeyAndJoinSide<K> makeOtherKey(final K
key, final long timestamp);
+
+ protected abstract VThis getThisValue(final LeftOrRightValue<? extends
VLeft, ? extends VRight> leftOrRightValue);
+
+ protected abstract VOther getOtherValue(final LeftOrRightValue<?
extends VLeft, ? extends VRight> leftOrRightValue);
+
private void emitNonJoinedOuterRecords(
- final KeyValueStore<TimestampedKeyAndJoinSide<K>,
LeftOrRightValue<V1, V2>> store,
- final Record<K, V1> record) {
+ final KeyValueStore<TimestampedKeyAndJoinSide<K>,
LeftOrRightValue<VLeft, VRight>> store,
+ final Record<K, ?> record) {
Review Comment:
nit:
```java
private void emitNonJoinedOuterRecords(final
KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<VLeft, VRight>>
store,
final Record<K, ?> record) {
```
##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java:
##########
@@ -275,20 +246,22 @@ private void emitNonJoinedOuterRecords(
}
}
- @SuppressWarnings("unchecked")
- private VOut getNullJoinedValue(
- final K key,
- final LeftOrRightValue<V1, V2> leftOrRightValue) {
- // depending on the JoinSide fill in the joiner key and joiner
values
- if (isLeftSide) {
- return joiner.apply(key,
- leftOrRightValue.getLeftValue(),
- leftOrRightValue.getRightValue());
- } else {
- return joiner.apply(key,
- (V1) leftOrRightValue.getRightValue(),
- (V2) leftOrRightValue.getLeftValue());
- }
+ private void forwardNonJoinedOuterRecords(final Record<K, ?> record,
final KeyValue<? extends TimestampedKeyAndJoinSide<K>, ? extends
LeftOrRightValue<VLeft, VRight>> nextKeyValue) {
Review Comment:
I think it would be simpler to define this method as:
```java
private void forwardNonJoinedOuterRecords(final Record<K, VThis> record,
final TimestampedKeyAndJoinSide<K>
timestampedKeyAndJoinSide,
final LeftOrRightValue<VLeft,
VRight> leftOrRightValue) {
```
It makes the code a bit simpler and shorter.
Also here, why not `Record<K, VThis> record`. That is exactly the type used
at the call site.
##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java:
##########
@@ -187,17 +157,25 @@ public void process(final Record<K, V1> record) {
context().forward(record.withValue(joiner.apply(record.key(), record.value(),
null)));
} else {
sharedTimeTracker.updatedMinTime(inputRecordTimestamp);
- outerJoinStore.ifPresent(store -> store.put(
- TimestampedKeyAndJoinSide.make(isLeftSide,
record.key(), inputRecordTimestamp),
- LeftOrRightValue.make(isLeftSide,
record.value())));
+ putInOuterJoinStore(record);
}
}
}
}
+ protected abstract TimestampedKeyAndJoinSide<K> makeThisKey(final K
key, final long inputRecordTimestamp);
+
+ protected abstract LeftOrRightValue<VLeft, VRight> makeThisValue(final
VThis thisValue);
+
+ protected abstract TimestampedKeyAndJoinSide<K> makeOtherKey(final K
key, final long timestamp);
+
+ protected abstract VThis getThisValue(final LeftOrRightValue<? extends
VLeft, ? extends VRight> leftOrRightValue);
+
+ protected abstract VOther getOtherValue(final LeftOrRightValue<?
extends VLeft, ? extends VRight> leftOrRightValue);
+
private void emitNonJoinedOuterRecords(
- final KeyValueStore<TimestampedKeyAndJoinSide<K>,
LeftOrRightValue<V1, V2>> store,
- final Record<K, V1> record) {
+ final KeyValueStore<TimestampedKeyAndJoinSide<K>,
LeftOrRightValue<VLeft, VRight>> store,
+ final Record<K, ?> record) {
Review Comment:
Shouldn't that be
```java
final Record<K, VThis> record
```
?
##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinSide.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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 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.state.internals.LeftOrRightValue;
+import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide;
+
+import java.util.Optional;
+
+class KStreamKStreamLeftJoinSide<K, VLeft, VRight, VOut> extends
KStreamKStreamJoin<K, VLeft, VRight, VOut, VLeft, VRight> {
+
+ KStreamKStreamLeftJoinSide(final String otherWindowName,
+ final JoinWindowsInternal windows,
+ final ValueJoinerWithKey<? super K, ? super
VLeft, ? super VRight, ? extends VOut> joiner,
+ final boolean outer,
+ final Optional<String> outerJoinWindowName,
+ final TimeTrackerSupplier
sharedTimeTrackerSupplier) {
+ super(otherWindowName, windows, joiner, outer, outerJoinWindowName,
windows.beforeMs, windows.afterMs,
+ sharedTimeTrackerSupplier);
+ }
+
+ @Override
+ public Processor<K, VLeft, K, VOut> get() {
+ return new KStreamKStreamJoinLeftProcessor();
+ }
+
+ private class KStreamKStreamJoinLeftProcessor extends
KStreamKStreamJoinProcessor {
+
+
Review Comment:
```suggestion
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]