Skip to content

Commit

Permalink
KAFKA-15774: refactor windowed stores to use StoreFactory (#14708)
Browse files Browse the repository at this point in the history
This is a follow up from #14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
  • Loading branch information
agavra authored Nov 11, 2023
1 parent 989618a commit 39cacca
Show file tree
Hide file tree
Showing 16 changed files with 553 additions and 515 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -535,7 +535,7 @@ public class StreamsConfig extends AbstractConfig {

public static final String ROCKS_DB = "rocksDB";
public static final String IN_MEMORY = "in_memory";
public static final String DEFAULT_DSL_STORE_DEFAULT = ROCKS_DB;
public static final String DEFAULT_DSL_STORE = ROCKS_DB;

/** {@code default.windowed.key.serde.inner} */
@SuppressWarnings("WeakerAccess")
Expand Down Expand Up @@ -1001,7 +1001,7 @@ public class StreamsConfig extends AbstractConfig {
CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC)
.define(DEFAULT_DSL_STORE_CONFIG,
Type.STRING,
DEFAULT_DSL_STORE_DEFAULT,
DEFAULT_DSL_STORE,
in(ROCKS_DB, IN_MEMORY),
Importance.LOW,
DEFAULT_DSL_STORE_DOC)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.internals.StreamsConfigUtils;
import org.apache.kafka.streams.kstream.internals.MaterializedInternal;
import org.apache.kafka.streams.processor.TimestampExtractor;

import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper;
Expand Down Expand Up @@ -218,7 +219,7 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo
}

public Materialized.StoreType parseStoreType() {
return Materialized.StoreType.parse(storeType);
return MaterializedInternal.parse(storeType);
}

public boolean isNamedTopology() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
import org.apache.kafka.streams.state.KeyValueStore;
Expand Down Expand Up @@ -71,17 +70,6 @@ public class Materialized<K, V, S extends StateStore> {
public enum StoreType {
ROCKS_DB,
IN_MEMORY;

public static StoreType parse(final String storeType) {
switch (storeType) {
case StreamsConfig.IN_MEMORY:
return StoreType.IN_MEMORY;
case StreamsConfig.ROCKS_DB:
return StoreType.ROCKS_DB;
default:
throw new IllegalStateException("Unexpected storeType: " + storeType);
}
}
}

private Materialized(final StoreSupplier<S> storeSupplier) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,7 @@
package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.common.utils.Bytes;
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;
import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
Expand All @@ -31,38 +28,17 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;

/**
* Materializes a key-value store as either a {@link TimestampedKeyValueStoreBuilder} or a
* {@link VersionedKeyValueStoreBuilder} depending on whether the store is versioned or not.
*/
public class KeyValueStoreMaterializer<K, V> implements StoreFactory {
public class KeyValueStoreMaterializer<K, V> extends MaterializedStoreFactory<K, V, KeyValueStore<Bytes, byte[]>> {
private static final Logger LOG = LoggerFactory.getLogger(KeyValueStoreMaterializer.class);

private final MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>> materialized;
private final Set<String> connectedProcessorNames = new HashSet<>();

private Materialized.StoreType defaultStoreType
= Materialized.StoreType.parse(StreamsConfig.DEFAULT_DSL_STORE_DEFAULT);

public KeyValueStoreMaterializer(
final MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>> materialized
) {
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
super(materialized);
}

@Override
Expand Down Expand Up @@ -127,21 +103,6 @@ public long historyRetention() {
return ((VersionedBytesStoreSupplier) materialized.storeSupplier()).historyRetentionMs();
}

@Override
public Set<String> connectedProcessorNames() {
return connectedProcessorNames;
}

@Override
public boolean loggingEnabled() {
return materialized.loggingEnabled();
}

@Override
public String name() {
return materialized.storeName();
}

@Override
public boolean isWindowStore() {
return false;
Expand All @@ -152,26 +113,4 @@ public boolean isVersionedStore() {
return materialized.storeSupplier() instanceof VersionedBytesStoreSupplier;
}

@Override
public Map<String, String> logConfig() {
return materialized.logConfig();
}

@Override
public StoreFactory withCachingDisabled() {
materialized.withCachingDisabled();
return this;
}

@Override
public StoreFactory withLoggingDisabled() {
materialized.withLoggingDisabled();
return this;
}

@Override
public boolean isCompatibleWith(final StoreFactory storeFactory) {
return (storeFactory instanceof KeyValueStoreMaterializer)
&& ((KeyValueStoreMaterializer<?, ?>) storeFactory).materialized.equals(materialized);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.common.serialization.Serde;
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.TopologyConfig;
Expand Down Expand Up @@ -59,6 +60,17 @@ public MaterializedInternal(final Materialized<K, V, S> materialized,
}
}

public static StoreType parse(final String storeType) {
switch (storeType) {
case StreamsConfig.IN_MEMORY:
return StoreType.IN_MEMORY;
case StreamsConfig.ROCKS_DB:
return StoreType.ROCKS_DB;
default:
throw new IllegalStateException("Unexpected storeType: " + storeType);
}
}

public String queryableStoreName() {
return queryable ? storeName() : null;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.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 {
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) {
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
}

@Override
public Set<String> connectedProcessorNames() {
return connectedProcessorNames;
}

@Override
public boolean loggingEnabled() {
return materialized.loggingEnabled();
}

@Override
public String name() {
return materialized.storeName();
}

@Override
public Map<String, String> logConfig() {
return materialized.logConfig();
}

@Override
public StoreFactory withCachingDisabled() {
materialized.withCachingDisabled();
return this;
}

@Override
public StoreFactory withLoggingDisabled() {
materialized.withLoggingDisabled();
return this;
}

@Override
public boolean isCompatibleWith(final StoreFactory storeFactory) {
return (storeFactory instanceof MaterializedStoreFactory)
&& ((MaterializedStoreFactory<?, ?, ?>) storeFactory).materialized.equals(materialized);
}
}
Loading

0 comments on commit 39cacca

Please sign in to comment.