guozhangwang commented on a change in pull request #10861:
URL: https://github.com/apache/kafka/pull/10861#discussion_r650448957



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
##########
@@ -76,18 +76,37 @@
 
     private final long graceMs;
 
+    protected final boolean enableSpuriousResultFix;
+
+    protected JoinWindows(final JoinWindows joinWindows) {

Review comment:
       How about moving this copy-constructor to JoinWindowsInternal?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
##########
@@ -82,20 +87,23 @@
 
     private class KStreamKStreamJoinProcessor extends AbstractProcessor<K, V1> 
{
         private WindowStore<K, V2> otherWindowStore;
-        private StreamsMetricsImpl metrics;
         private Sensor droppedRecordsSensor;
         private Optional<WindowStore<KeyAndJoinSide<K>, LeftOrRightValue>> 
outerJoinWindowStore = Optional.empty();
 
-        @SuppressWarnings("unchecked")
         @Override
         public void init(final ProcessorContext context) {
             super.init(context);
-            metrics = (StreamsMetricsImpl) context.metrics();
+            final StreamsMetricsImpl metrics = (StreamsMetricsImpl) 
context.metrics();
             droppedRecordsSensor = 
droppedRecordsSensor(Thread.currentThread().getName(), 
context.taskId().toString(), metrics);
             otherWindowStore = context.getStateStore(otherWindowName);
 
-            if 
(StreamsConfig.InternalConfig.getBoolean(context().appConfigs(), 
ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) {
-                outerJoinWindowStore = outerJoinWindowName.map(name -> 
context.getStateStore(name));
+            if (enableSpuriousResultFix

Review comment:
       Thinking a bit more to not evaluate the condition twice: since we call 
`addStateStore` in the logical plan, a.k.a. `StreamStreamJoinNode` first if 
both conditions are met, could we just blindly try to get the store from 
`context.getStateStore()` here, which would return `null` if the store was not 
added during logical plan and hence does not exist anyways. If it is `null` we 
can still convert it to an `Optional.empty()`.




-- 
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.

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


Reply via email to