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


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreFactory.java:
##########
@@ -66,6 +66,10 @@ default void configure(final StreamsConfig config) {
 
     boolean isVersionedStore();
 
+    // TODO: consider moving all the log configuration code 
(InternalTopicConfig)

Review Comment:
   this would be a great cleanup, I'm definitely not a fan of the 
`#isWindowStore` and `retentionPeriod` logic which kind of breaks the 
abstraction. 
   
   Just wondering, would we also be able to get rid of the `#historyRetention` 
and `#isVersionedStore` or is that used for something else?



##########
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:
   Thanks!



##########
streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java:
##########
@@ -216,12 +232,38 @@ public TopologyConfig(final String topologyName, final 
StreamsConfig globalAppCo
         } else {
             storeType = globalAppConfigs.getString(DEFAULT_DSL_STORE_CONFIG);
         }
+
+        if (isTopologyOverride(DSL_STORE_SUPPLIERS_CLASS_CONFIG, 
topologyOverrides)) {
+            dslStoreSuppliers = getString(DSL_STORE_SUPPLIERS_CLASS_CONFIG);
+            log.info("Topology {} is overriding {} to {}", topologyName, 
DSL_STORE_SUPPLIERS_CLASS_CONFIG, dslStoreSuppliers);
+        } else {
+            dslStoreSuppliers = 
globalAppConfigs.getString(DSL_STORE_SUPPLIERS_CLASS_CONFIG);
+        }
     }
 
+    @Deprecated
     public Materialized.StoreType parseStoreType() {
         return MaterializedInternal.parse(storeType);
     }
 
+    /**
+     * @return the DslStoreSuppliers if the value was explicitly configured 
(either by
+     *         {@link StreamsConfig#DEFAULT_DSL_STORE} or {@link 
StreamsConfig#DSL_STORE_SUPPLIERS_CLASS_CONFIG})
+     */
+    public Optional<DslStoreSuppliers> resolveDslStoreSuppliers() {

Review Comment:
   Ah....in that case I would say just do whatever is easiest and cleanest for 
now (so presumably, just making this public)
   
   If this class was intended to be a real and lasting public interface then 
it's probably neater to just have `isTopologyOverride` and `isGlobalOverride` 
as public APIs and then do the actual DslStoreSupplier resolution in an 
internal utility class. But I think we'll want to just deprecate/remove this 
entire class soon anyways, so it doesn't really hurt to have one more ugly API 
if we consider it effectively temporary. Maybe Matthias can even tackle this 
(deprecating TopologyConfig) in the KIP he was planning to clean up similar 
ugly APIs over in StreamsConfig -- I'll bring it up with him.
   
   FYI though it's probably worth defending this choice when you update the KIP 
thread, with some form of the above argument. It's definitely not a good public 
interface but the whole thing needs to go and we can deprecate it all together



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreFactory.java:
##########
@@ -66,6 +66,10 @@ default void configure(final StreamsConfig config) {
 
     boolean isVersionedStore();
 
+    // TODO: consider moving all the log configuration code 
(InternalTopicConfig)

Review Comment:
   Btw unless you plan on doing it yourself in an immediate followup PR, please 
remove the TODO and file a JIRA ticket for this. As a general AK practice we 
don't leave inline TODOs unless they help to understand why we are currently 
doing something in a particular way (ie if it also makes sense as a non-TODO 
code comment then it's probably a good TODO as well)



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