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


##########
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:
   @ableegoldman - I'll add this back to the KIP, LMK if you think this API 
makes sense since it is technically public



##########
streams/src/main/java/org/apache/kafka/streams/state/BuiltinDslStoreSuppliers.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.state;
+
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import 
org.apache.kafka.streams.state.internals.RocksDbIndexedTimeOrderedWindowBytesStoreSupplier;
+import 
org.apache.kafka.streams.state.internals.RocksDbTimeOrderedSessionBytesStoreSupplier;
+
+public class BuiltinDslStoreSuppliers {
+
+    public static final DslStoreSuppliers ROCKS_DB = new 
RocksDbDslStoreSuppliers();
+    public static final DslStoreSuppliers IN_MEMORY = new 
InMemoryDslStoreSuppliers();
+
+    public static class RocksDbDslStoreSuppliers implements DslStoreSuppliers {

Review Comment:
   made the change, but FWIW it doesn't seem consistent throughout the code 
base 
   
![image](https://github.com/apache/kafka/assets/3172405/b2a083ab-9f33-4044-aefc-60ed1b3b49ee)
   



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/MaterializedStoreFactory.java:
##########
@@ -16,42 +16,20 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
-import org.apache.kafka.streams.StreamsConfig;
-import org.apache.kafka.streams.kstream.Materialized;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.internals.StoreFactory;
 
 /**
  * {@code MaterializedStoreFactory} is the base class for any {@link 
StoreFactory} that
  * wraps a {@link MaterializedInternal} instance.
  */
-public abstract class MaterializedStoreFactory<K, V, S extends StateStore> 
implements StoreFactory {
+public abstract class MaterializedStoreFactory<K, V, S extends StateStore> 
extends AbstractConfigurableStoreFactory {
     protected final MaterializedInternal<K, V, S> materialized;
-    private final Set<String> connectedProcessorNames = new HashSet<>();
-    protected Materialized.StoreType defaultStoreType
-            = MaterializedInternal.parse(StreamsConfig.DEFAULT_DSL_STORE);
 
     public MaterializedStoreFactory(final MaterializedInternal<K, V, S> 
materialized) {
+        super(materialized.dslStoreSuppliers().orElse(null));
         this.materialized = materialized;
-
-        // this condition will never be false; in the next PR we will
-        // remove the initialization of storeType from MaterializedInternal
-        if (materialized.storeType() != null) {
-            defaultStoreType = materialized.storeType;
-        }
-    }
-
-    @Override
-    public void configure(final StreamsConfig config) {
-        // in a follow-up PR, this will set the defaultStoreType to the 
configured value

Review Comment:
   this PR does the follow-up in `AbstractConfiguredStoreFactory`



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java:
##########
@@ -265,31 +261,6 @@ private void assertUniqueStoreNames(final 
WindowBytesStoreSupplier supplier,
         }
     }
 
-    private static <K, V> StoreBuilder<WindowStore<K, V>> 
joinWindowStoreBuilder(final String storeName,

Review Comment:
   All this deleted code was moved to their own StoreFactory implementations



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