ableegoldman commented on code in PR #14648:
URL: https://github.com/apache/kafka/pull/14648#discussion_r1396507648


##########
streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java:
##########
@@ -16,17 +16,21 @@
  */
 package org.apache.kafka.streams;
 
+import java.util.Optional;

Review Comment:
   nit: out of order



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java:
##########
@@ -227,9 +223,9 @@ public StreamStreamJoinNode<K, V1, V2, VR> build() {
                                               joinMergeProcessorParameters,
                                               
thisWindowedStreamProcessorParameters,
                                               
otherWindowedStreamProcessorParameters,
-                                              thisWindowStoreBuilder,
-                                              otherWindowStoreBuilder,
-                                              outerJoinWindowStoreBuilder,
+                                              thisStoreFactory,
+                    otherStoreFactory,

Review Comment:
   nit: formatting



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/OuterStreamJoinStoreFactory.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
+import static 
org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration;
+
+import java.time.Duration;
+import java.util.Map;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.kstream.JoinWindows;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.internals.StoreFactory;
+import org.apache.kafka.streams.state.DslKeyValueParams;
+import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.Stores;
+import 
org.apache.kafka.streams.state.internals.InMemoryWindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.internals.LeftOrRightValue;
+import org.apache.kafka.streams.state.internals.LeftOrRightValueSerde;
+import org.apache.kafka.streams.state.internals.ListValueStoreBuilder;
+import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide;
+import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSideSerde;
+
+public class OuterStreamJoinStoreFactory<K, V1, V2> extends 
AbstractConfigurableStoreFactory {
+
+    private final String name;
+    private final StreamJoinedInternal<K, V1, V2> streamJoined;
+    private final JoinWindows windows;
+    private boolean loggingEnabled;
+
+    public enum Type {
+        RIGHT,
+        LEFT
+    }
+
+    public OuterStreamJoinStoreFactory(
+            final String name,
+            final StreamJoinedInternal<K, V1, V2> streamJoined,
+            final JoinWindows windows,
+            final Type type
+    ) {
+        super(streamJoined.dslStoreSuppliers());
+        this.name = buildOuterJoinWindowStoreName(streamJoined, name, type) + 
"-store";
+        this.streamJoined = streamJoined;
+        this.windows = windows;
+        this.loggingEnabled = streamJoined.loggingEnabled();
+    }
+
+    @Override
+    public StateStore build() {
+        final Duration retentionPeriod = Duration.ofMillis(retentionPeriod());
+        final Duration windowSize = Duration.ofMillis(windows.size());
+        final String rpMsgPrefix = 
prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod");
+        final long retentionMs = validateMillisecondDuration(retentionPeriod, 
rpMsgPrefix);
+        final String wsMsgPrefix = prepareMillisCheckFailMsgPrefix(windowSize, 
"windowSize");
+        final long windowSizeMs = validateMillisecondDuration(windowSize, 
wsMsgPrefix);
+
+        if (retentionMs < 0L) {
+            throw new IllegalArgumentException("retentionPeriod cannot be 
negative");
+        }
+        if (windowSizeMs < 0L) {
+            throw new IllegalArgumentException("windowSize cannot be 
negative");
+        }
+        if (windowSizeMs > retentionMs) {
+            throw new IllegalArgumentException("The retention period of the 
window store "
+                    + name + " must be no smaller than its window size. Got 
size=["
+                    + windowSizeMs + "], retention=[" + retentionMs + "]");
+        }
+
+        final TimestampedKeyAndJoinSideSerde<K> timestampedKeyAndJoinSideSerde 
= new TimestampedKeyAndJoinSideSerde<>(streamJoined.keySerde());
+        final LeftOrRightValueSerde<V1, V2> leftOrRightValueSerde = new 
LeftOrRightValueSerde<>(streamJoined.valueSerde(), 
streamJoined.otherValueSerde());
+
+        // TODO: we should allow for configuration of this store explicitly 
instead of assuming that it should
+        // share the same type of store as thisStoreSupplier
+        final boolean useInMemoryStore = streamJoined.thisStoreSupplier() != 
null
+                && streamJoined.thisStoreSupplier() instanceof 
InMemoryWindowBytesStoreSupplier;
+        final KeyValueBytesStoreSupplier supplier = useInMemoryStore
+                ? Stores.inMemoryKeyValueStore(name)
+                : dslStoreSuppliers().keyValueStore(new 
DslKeyValueParams(name));

Review Comment:
   I think this is going to resolve to the wrong store type in two cases where 
it should, at least in theory, be possible to select the right one:
   
   1) Users sets the global `dsl.store.suppliers` (ie the config) to an 
in-memory or custom store, but want to fix this stream-stream join to rocksdb. 
So `streamJoined.storeSupplier` == RocksdbBytesStoreSupplier
   2) User sets a global `dsl.store.suppliers` and does not configure the 
StreamJoined with a `dslStoreSupplier`
   
   Frankly 1) is probably not a big deal, but 2) seems like a more serious 
violation. It's stemming from the fact that we always initialize the 
`dslStoreSuppliers` in StreamJoined to the default 
`Materialized.StoreType.ROCKS_DB` if not explicitly configured. So the 
`streamJoined#dslStoreSuppliers` that we pass into the 
`AbstractConfigurableStoreFactory` constructor is never null, which means we'll 
always hit the `Materialized.StoreType.ROCKS_DB` and won't respect the user's 
global config
   
   At least, that's my reading of the situation...it's a bit complex so maybe I 
missed something.
   
   Seems like we'll need to either use a flag to mark when the user has 
explicitly set the StreamJoined's `dslStoreSuppliers`, or just leave it null if 
the user hasn't set it. The latter feels like the right thing to do but of 
course that has implications for the rest of the code. Can we make it work 
without initializing to the default RocksdbDslStoreSuppliers? 
   
   You know the code better than me so I'll leave that up to you, hopefully I 
got the point across here. At the very least, I guess we could just treat the 
RocksDB as a special case, and ignore the `StreamJoined#dslStoreSuppliers` in 
favor of the global default if it's equal to `Materialized.StoreType.ROCKS_DB`



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/StreamJoinedStoreFactory.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.JoinWindows;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.internals.StoreFactory;
+import org.apache.kafka.streams.state.DslWindowParams;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.WindowStore;
+
+public class StreamJoinedStoreFactory<K, V1, V2> extends 
AbstractConfigurableStoreFactory {
+
+    private final String name;
+    private final JoinWindows windows;
+    private final Serde<?> valueSerde;
+    private final WindowBytesStoreSupplier storeSupplier;
+    private final StreamJoinedInternal<K, V1, V2> joinedInternal;
+
+    private boolean loggingEnabled;
+    private final Map<String, String> logConfig;
+
+    public enum Type {
+        THIS,
+        OTHER
+    }
+
+    public StreamJoinedStoreFactory(
+            final String name,
+            final JoinWindows windows,
+            final StreamJoinedInternal<K, V1, V2> joinedInternal,
+            final Type type
+    ) {
+        super(joinedInternal.dslStoreSuppliers());
+        this.name = name + "-store";
+        this.joinedInternal = joinedInternal;
+        this.windows = windows;
+        this.loggingEnabled = joinedInternal.loggingEnabled();
+        this.logConfig = new HashMap<>(joinedInternal.logConfig());
+
+        // since this store is configured to retain duplicates, we should
+        // not compact, so we override the configuration to make sure that
+        // it's just delete (window stores are configured to compact,delete)
+        this.logConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_DELETE);

Review Comment:
   Is this new logic? I don't think it hurts anything to have compaction 
enabled...of course it's also not doing anything so I guess this optimization 
makes sense



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

Reply via email to