Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: Under Discussion Adopted (1.0)

Discussion thread: [DISCUS] KIP-182: Reduce Streams DSL overloads and allow easier use of custom storage engineshere [Change the link from the KIP proposal email archive to your own email thread]

JIRA: KAFKA-5651 

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

Code Block
languagejava
titleKStream
void print(final Printed<K, V> printed);

KStream<K, V> through(final String topic, final Produced<K, V> partitioned);

void to(final String topic, final Produced<V, V> partitioned);

KGroupedStream<K, V> groupByKey(final Serialized<K, V> serialized);

<KR> KGroupedStream<KR, V> groupBy(final KeyValueMapper<? super K, ? super V, KR> selector, Serialized<KR, V> serialized);

<VO, VR> KStream<K, VR> join(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final Joined<K, V, VO> options);

<VT, VR> KStream<K, VR> join(final KTable<K, VT> other, final ValueJoiner<? super V, ? super VT, ? extends VR> joiner, final Joined<K, V, VT> options);

<VO, VR> KStream<K, VR> leftJoin(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final Joined<K, V, VO> options);

<VT, VR> KStream<K, VR> leftJoin(final KTable<K, VT> other, final ValueJoiner<? super V, ? super VT, ? extends VR> joiner, final JoinWindows windows, final Joined<K, V, VT> options);

<VO, VR> KStream<K, VR> outerJoin(final KStream<K, VO> other, final ValueJoiner<? super V, ? super VO, ? extends VR> joiner, final JoinWindows windows, final Joined<K, V, VO> options);

<VT


Code Block
languagejava
titleKTable
<KR, VR> KStream<KKGroupedTable<KR, VR> outerJoingroupBy(final KTable<K,KeyValueMapper<? VT>super otherK, final ValueJoiner<? super V, ?KeyValue<KR, superVR>> VTselector, ?Serialized<KR, extends VR> joiner, final JoinWindows windows, final Joined<K, V, VT> options);
Code Block
languagejava
titleKTable
<KR, VR> KGroupedTable<KR, VR> groupBy(final KeyValueMapper<serialized);

KTable<K, V> filter(final Predicate<? super K, ? super V> Vpredicate, final KeyValue<KRMaterialized<K, VR>> selectorV, Serialized<KRKeyValueStore<Bytes, VR> serializedbyte[]>> materialized);

KTable<K, V> filterfilterNot(final Predicate<? super K, ? super V> predicate, final Materialized<K, V, KeyValueStore<KKeyValueStore<Bytes[], V>>byte[]>> materialized);

<VR> KTable<K, V>VR> filterNotmapValues(final Predicate<ValueMapper<? super KV, ? superextends V>VR> predicatemapper, final Materialized<K, V, KeyValueStore<KKeyValueStore<Bytes[], V>>byte[]>> materialized);

<VR><VO, VR> KTable<K, VR> mapValuesjoin(final ValueMapper<?KTable<K, superVO> Vother,
 ?    extends VR> mapper, final Materialized<K, V, KeyValueStore<K, V>> materialized);

void to(final String topic, final Produced<V, V> options);

KTable<K, V> through(final String topic, final Materialized<K, V> options);

<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
                            final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
                            final Materialized<K, VR, KeyValueStore<KKeyValueStore<Bytes, VR>>byte[]>> materialized);

<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
                                final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
                                final Materialized<K, VR, KeyValueStore<KKeyValueStore<Bytes, VR>>byte[]>> materialized);

<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
                                 final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
                                 final Materialized<K, VR, KeyValueStore<KKeyValueStore<Bytes, VR>>byte[]>> materialized);
 

 

We add some new helper methods to Stores so people can conveniently and quickly create basic StateStoreSuppliers for use in the DSL or PAPI. We will also deprecate the existing Stores.create(...)

Code Block
languagejava
titleStores
public static <K, V> BytesStoreSupplier<KeyValueStore<Bytes, byte[]>> KeyValueBytesStoreSupplier persistentKeyValueStore(final String name) 

public static <K, V> BytesStoreSupplier<KeyValueStore<Bytes, byte[]>>KeyValueBytesStoreSupplier inMemoryKeyValueStore(final String name)

public static <K, V> BytesStoreSupplier<KeyValueStore<Bytes, byte[]>>KeyValueBytesStoreSupplier lruMap(final String name) 

public static <K, V> BytesStoreSupplier<WindowStore<Bytes, byte[]>> WindowBytesStoreSupplier persistentWindowStore(final String name,  
																	final Windows windows)

public static <K, V> BytesStoreSupplier<SessionStore<Bytes, byte[]>> persistentSessionStore(final String name,
                                                                              final SessionWindows windows) 

long retentionPeriod, 
																	final int numSegments, 
																	final long windowSize, 
																	final boolean retainDuplicates)
public static <K, V> SessionBytesStoreSupplier persistentSessionStore(final String name, final long retentionPeriod)
/**
 *  The following methods are for use with the PAPI. They allow building of StateStores that can be wrapped with
 *  caching, logging, and any other convenient wrappers provided by the KafkaStreams library
 */ 
public <K, V> StateStoreBuilder<WindowStore<K, V>> windowStoreBuilder(final BytesStoreSupplier<WindowStore<Bytes, byte[]>>WindowBytesStoreSupplier supplier, 
																	  final Serde<K> keySerde, 
																      final Serde<V> valueSerde)

public <K, V> StateStoreBuilder<KeyValueStore<K, V>> keyValueStoreBuilder(final BytesStoreSupplier<KeyValueStore<Bytes, byte[]>> KeyValueBytesStoreSupplier supplier,
 																		  final Serde<K> keySerde, 
																          final Serde<V> valueSerde)

public <K, V> StateStoreBuilder<SessionStore<K, V>> sessionStoreBuilder(final BytesStoreSupplier<SessionStore<Bytes, byte[]>> SessionBytesStoreSupplier supplier,
																	    final Serde<K> keySerde, 
																        final Serde<V> valueSerde)

...

Code Block
languagejava
titleTopologyBuilderTopology
public synchronized <K, V> TopologyBuilderTopology addGlobalStore(final StateStoreBuilder storeSupplier,
                                                          final String sourceName,
                                                          final TimestampExtractor timestampExtractor,
                                                          final Deserializer keyDeserializer,
                                                          final Deserializer valueDeserializer,
                                                          final String topic,
                                                          final String processorName,
                                                          final ProcessorSupplier stateUpdateSupplier)
 
public synchronized final TopologyBuilderTopology addStateStore(final StateStoreBuilder supplier, final String... processorNames)
 

...

Code Block
languagejava
titleKGroupedStream
<W extends Window> WindowedKStream<KTimeWindowedKStream<K, V> windowedBy(Windows<W> timeWindows);

SessionWindowedKStream<K, V> sessionWindowedBywindowedBy(SessionWindows sessionWindows);

KTable<K, Long> count(final Materialized<K, Long> materialized);

KTable<K, V> reduce(final Reducer<V> reducer, final Materialized<K, V, KeyValueStore<KKeyValueStore<Bytes, V>>byte[]>> materialized);

<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final Aggregator<? super K, ? super V, VR> aggregator,
                             final Serde<VR>Materialized<K, aggValueSerdeVR,
 KeyValueStore<Bytes, byte[]>> materialized);


Code Block
languagejava
titleKGroupedTable
KTable<K, Long> count(final Materialized<K, V, KeyValueStore<Bytes,                     final Materialized<K, VR, KeyValueStore<K, VR>> materialized);
Code Block
languagejava
titleKGroupedTable
KTable<K, Long> count(final Materialized<K, V, KeyValueStore<K, V>> materialized);byte[]>> materialized);

KTable<K, V> reduce(final Reducer<V> adder, final Reducer<V> subtractor, final Materialized<K, V, KeyValueStore<KKeyValueStore<Bytes, V>>byte[]>> materialized);

<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                             final Aggregator<? super K, ? super V, VR> aggregator,
                             final Aggregator<? super K, ? super V, VR> subtractor,
                             final Materialized<K, VR, KeyValueStore<KKeyValueStore<Bytes, VR>>byte[]>> materialized);

 

...

For StreamsBuilder we remove all stream, table, and globalTable overloads that take more than a single argument and replace them with:

Code Block
languagejava
titleWindowedKStreamStreamsBuilder
public interfacesynchronized WindowedKStream<K<K, V> {

KStream<K, V> stream(final String topic)
 
public synchronized <K, V> KStream<K, V> stream(final String topic, final Consumed<K, V> options)
 
public synchronized <K, V> KStream<K, V> stream(final Collection<String> topic, final Consumed<K, V> options)
 
public synchronized <K, V> KStream<K, V> stream(final Collection<String> topic)

public synchronized <K, V> KStream<K, V> stream(final Pattern pattern, final Consumed<K, V> options)

public synchronized <K, V> KTable<K, V> table(final String topic, final Consumed<K, V> consumed)

public synchronized <K, V> KTable<K, V> table(final String topic, final Consumed<K, V> consumed, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized)
 
public synchronized <K, V> KTable<K, V> table(final String topic, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized)

public synchronized <K, V> GlobalKTable<K, V> globalTable(final String topic, final Consumed<K, V> consumed)

public synchronized <K, V> GlobalKTable<K, V> globalTable(final String topic, final Consumed<K, V> consumed, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized)
 
public synchronized <K, V> GlobalKTable<K, V> globalTable(final String topic, final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized)

 


New classes and interfaces:

Code Block
languagejava
titleWindowedKStream
public interface TimeWindowedKStream<K, V> {

    KTable<Windowed<K>, Long> count();

    KTable<Windowed<K>, Long> count(final Materialized<K, Long, WindowStore<Bytes, byte[]>> materializedAs);

    <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
 KTable<Windowed<K>, Long> count();

    KTable<Windowed<K>, Long> count(final Materialized<K, Long, WindowStore<K, Long>> materializedAs);

    <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                                           final Aggregator<? super K, ? super V, VR> aggregator);

    <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                                           final Aggregator<? super K, ? super V, VR> aggregator,
                                           final Materialized<K, VR, WindowStore<K, VR>> materializedAs                        final Aggregator<? super K, ? super V, VR> aggregator);


    <VR> KTable<Windowed<K>, V>VR> reduceaggregate(final Reducer<V> reducer);

Initializer<VR> initializer,
     KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                  final Aggregator<? super Materialized<KK, ? super V, WindowStore<K, V>> materializedAs);


} 
Code Block
languagejava
titleSessionWindowedKStream
public interface SessionWindowedKStream<K, V> {

VR> aggregator,
    KTable<Windowed<K>, Long> count();

    KTable<Windowed<K>, Long> count(final Materialized<K, Long, SessionStore<K, Long>> materializedAs);

    <VR, T> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                final Materialized<K, VR, WindowStore<Bytes, byte[]>> materializedAs);


    KTable<Windowed<K>, V> reduce(final Reducer<V> reducer);

    KTable<Windowed<K>,             final Aggregator<? super K, ? super V, VR> aggregator,
           V> reduce(final Reducer<V> reducer,
                                   final Materialized<K, V, WindowStore<Bytes, byte[]>> materializedAs);


} 


Code Block
languagejava
titleSessionWindowedKStream
public interface SessionWindowedKStream<K, V> {

    KTable<Windowed<K>, Long> count();

    KTable<Windowed<K>, Long> count(final Materialized<K, Long, SessionStore<Bytes, byte[]>> materializedAsMerger<? super K, T> sessionMerger);

    <VR, T> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                                              final Aggregator<? super K, ? super V, VR> aggregator,
                                              final Merger<? super K, T> sessionMerger,);

    <VR, T> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                                              final Materialized<K, VR, SessionStore<KAggregator<? super K, VR>> materializedAs);


    KTable<Windowed<K>, V> reduce(final Reducer<V> reducer);

? super V, VR> aggregator,
        KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                  final Merger<? super Materialized<KK, V,T> SessionStore<KsessionMerger,
 V>> materializedAs);
}
Code Block
languagejava
titleMaterialized
/**
 * Used when materializing a state store
 */
public class Materialized<K, V, S extends StateStore> {
                       public static <K, V, S extends StateStore> Materialized<K, V, S> as(final String storeName)

    public static <K, V, S extends StateStore>final Materialized<K, VVR, S> as(final StateStoreSupplier<S> supplier)

SessionStore<Bytes, byte[]>> materializedAs);


    public Materialized<KKTable<Windowed<K>, V, S> withValueSerdeV> reduce(final Serde<V>Reducer<V> valueSerdereducer);

    public Materialized<KKTable<Windowed<K>, V, S> withKeySerdeV> reduce(final Serde<K>Reducer<V> valueSerde)reducer,

    public Materialized<K, V, S> withLoggingEnabled(final Map<String, String> topicConfig)

    public Materialized<K, V, S> withLoggingDisabled()

    public Materialized<K, V, S> withCachingEnabled()

        publicfinal Materialized<K, V, S> withCachingDisabled()

} SessionStore<Bytes, byte[]>> materializedAs);
}


Code Block
languagejava
titleSerializedMaterialized
/**
 * Optional params that can be passed to groupBy and groupByKey Used when materializing a state store, i.e, during an aggregation operation or KTable operations
 */
public class Serialized<K, V>Materialized<K, V, S extends StateStore> {

    	public static <K, V>V, Serialized<K,S V>extends with(final Serde<K> keySerde, final Serde<V> valueSerdeStateStore> Materialized<K, V, S> as(final String storeName)

    public static <K, V> Serialized<KMaterialized<K, V> with(final Serde<K> keySerde, final Serde<V> valueSerde, final Serde<V> otherValueSerde)
V, WindowStore<Bytes, byte[]>> as(final WindowBytesStoreSupplier supplier);
 
    public Serialized<Kstatic <K, V> withKeySerde Materialized<K, V, SessionStore<Bytes, byte[]>> as(final Serde<K>SessionBytesStoreSupplier keySerdesupplier);
 
    public Serialized<Kstatic <K, V> withValueSerde(final Serde valueSerde)
}
Code Block
languagejava
titleJoined
/**
 * Optional params that can be passed to join, leftJoin, outerJoin operations
 */
public class Joined<K, V, VO> {Materialized<K, V, KeyValueStore<Bytes, byte[]>> as(final KeyValueBytesStoreSupplier supplier)
 
    public Materialized<K, V, S> withValueSerde(final Serde<V> valueSerde)

    public static <KMaterialized<K, V, VO> Joined<K, V, VO> withS> withKeySerde(final Serde<K> keySerde, final Serde <V> valueSerde, final Serde<VO> otherValueSerde)

    public static <KMaterialized<K, V, VO> Joined<K, VS> withLoggingEnabled(final Map<String, VO> keySerde(final Serde<K> keySerdeString> topicConfig)

    public static <KMaterialized<K, V, VO> Joined<K S> withLoggingDisabled()

    public Materialized<K, V, VO>S> valueSerdewithCachingEnabled(final Serde<V> valueSerde)

    public static <KMaterialized<K, V, VO> Joined<K, V, VO> otherValueSerde(final Serde<V> valueSerde)

    public Joined<K, V, VO> withKeySerde(finalS> withCachingDisabled()

	public String storeName();

	public StoreSupplier<S> storeSupplier()

	public Serde<K> keySerde()

	public Serde<V> valueSerde()

	public boolean loggingEnabled()

	public Joined<KMap<String, V, VO> withValueSerde(final Serde<V> valueSerdeString> logConfig()

    	public Joined<K, V, VO> withOtherValueSerde(final Serde<VO> otherValueSerde)
}boolean cachingEnabled()

} 


Code Block
languagejava
titleProducedSerialized
/**
 * Optional argumentsparams that can be specifiedpassed whento doinggroupBy to and throughgroupByKey operations
 */
public staticclass <KSerialized<K, V> Produced<K,{

 V> with(final Serde<K> keySerde, final Serde<V> valueSerde)

public static <K, V> Produced<KSerialized<K, V> with(final StreamPartitioner<K, V> partitioner, final Serde<K> keySerde, final Serde<V> valueSerde) 

public  static <K, V>public Produced<KSerialized<K, V> keySerdewithKeySerde(final Serde<K> keySerde)

public static  <K, V>public Produced<KSerialized<K, V> valueSerdewithValueSerde(final Serde<V>Serde valueSerde)
 

	public static <K, V> Produced<K, V> streamPartitioner(final StreamPartitioner<K, V> partitioner) 

public Produced<K, V> withStreamPartitioner(final StreamPartitioner<K, V> partitioner) 
public Produced<K, V> withValueSerde(final Serde<V> valueSerde) 
public Produced<K, V> withKeySerde(final Serde<K> keySerde)
Code Block
languagejava
titlePrinted
Serde<K> keySerde()
 
    public Serde<V> valueSerde()
}


Code Block
languagejava
titleJoined
/**
 * Optional params that can be passed to join, leftJoin, outerJoin operations
 */
public class Joined<K, V, VO> {

    public static <K, V, VO> Joined<K, V, VO> with(final Serde<K> keySerde, final Serde <V> valueSerde, final Serde<VO> otherValueSerde)
 
/**
 * options that can be used when printing to stdout our writing to a file
 */
public class Printed<K, V> {
    public static <K, V> Printed<KV, VO> Joined<K, V, V>VO> toFilekeySerde(final StringSerde<K> filepathkeySerde)

    public static <K, V> Printed<KV, VO> Joined<K, V, V>VO> toSysOutvalueSerde(final Serde<V> valueSerde)

    public Printed<K static <K, V, VO> Joined<K, V, V>VO> withLabelotherValueSerde(final StringSerde<V> labelvalueSerde)

    private Printed<Kpublic Joined<K, V, V>VO> withFilewithKeySerde(final StringSerde<K> filepathkeySerde)

    public Printed<KJoined<K, V, V>VO> withKeySerdewithValueSerde(final Serde<K>Serde<V> keySerdevalueSerde)

    public Joined<K, Printed<KV, V>VO> withValueSerdewithOtherValueSerde(final Serde<V>Serde<VO> keySerdeotherValueSerde)
 
    	public Printed<K, V> withKeyValueMapper(final KeyValueMapper<? super K, ? super V, String> mapper)

}
Serde<K> keySerde()
 
    public Serde<V> valueSerde()
 
	public Serde<VO> otherValueSerde()
}


Code Block
languagejava
titleStateStoreBuilderProduced
/**
 * Optional Implementationsarguments ofthat thiscan willbe providespecified thewhen abilitydoing to wrapand athrough given StateStoreoperations
 * with or without caching/loggging etc.
public */
public interface StateStoreBuilder<T extends StateStore> {

    StateStoreBuilder<T> withCachingEnabled();
    StateStoreBuilder<T> withLoggingEnabled(Map<String, String> config);
    T build();
}
Code Block
languagejava
titleBytesStoreSupplier
public interface BytesStoreSupplier<T extends StateStore> {

    /**
     * Return the name of this state store supplier.
     * This must be a valid Kafka topic name; valid characters are ASCII alphanumerics, '.', '_' and '-'
     *
     * @return the name of this state store supplier
     */
    String name();

    /**
     * Return a new {@link StateStore} instance.
     *
     * @return a new {@link StateStore} instance of type T
     */
    T get();

}

 

Proposed Changes

static <K, V> Produced<K, V> with(final Serde<K> keySerde, final Serde<V> valueSerde)

public static <K, V> Produced<K, V> with(final Serde<K> keySerde, final Serde<V> valueSerde, final StreamPartitioner<K, V> partitioner) 

public static <K, V> Produced<K, V> keySerde(final Serde<K> keySerde)

public static <K, V> Produced<K, V> valueSerde(final Serde<V> valueSerde) 

public static <K, V> Produced<K, V> streamPartitioner(final StreamPartitioner<K, V> partitioner) 

public Produced<K, V> withStreamPartitioner(final StreamPartitioner<K, V> partitioner) 
public Produced<K, V> withValueSerde(final Serde<V> valueSerde) 
public Produced<K, V> withKeySerde(final Serde<K> keySerde)
public Serde<K> keySerde()
public Serde<K> valueSerde()
public StreamPartitioner<K, V> streamPartitioner()


Code Block
languagejava
titlePrinted
 
/**
 * options that can be used when printing to stdout our writing to a file
 */
public class Printed<K, V> {
    public static <K, V> Printed<K, V> toFile(final String filepath)

    public static <K, V> Printed<K, V> toSysOut()

    public Printed<K, V> withLabel(final String label)

    public Printed<K, V> withKeyValueMapper(final KeyValueMapper<? super K, ? super V, String> mapper)

    public ProcessorSupplier<K, V> build(final String processorName)
}


Code Block
languagejava
titleConsumed
/**
 * Options for consuming a topic as a KStream or KTable
 */
public class Consumed<K, V> {
    public static <K, V>  Consumed<K, V> with(final Serde<K> keySerde, final Serde<V> valueSerde, final TimestampExtractor extractor, final Topology.AutoOffsetReset resetPolicy)
    public static <K, V>  Consumed<K, V> with(final Serde<K> keySerde, final Serde<V> valueSerde)
    public static <K, V>  Consumed<K, V> with(final TimestampExtractor extractor)
    public static <K, V>  Consumed<K, V> with(final Topology.AutoOffsetReset resetPolicy)
    
    public Consumed<K, V> withKeySerde(final Serde<K> keySerde)

    public Consumed<K, V> withValueSerde(final Serde<V> valueSerde)

    public Consumed<K, V> withTimestampExtractor(final TimestampExtractor timestampExtractor)

    public Consumed<K, V> withOffsetResetPolicy(final Topology.AutoOffsetReset resetPolicy)
 
	public Serde<K> keySerde()
 
	public Serde<V> valueSerde()
 
	public TimestampExtractor timestampExtractor()
 
	public Toploogy.AutoOffsetReset offsetResetPolicy()
}


Code Block
languagejava
titleStateStoreBuilder
/**
 * Implementations of this will provide the ability to wrap a given StateStore
 * with or without caching/loggging etc.
 */
public interface StoreBuilder<T extends StateStore> {

    StoreBuilder<T> withCachingEnabled();
    StoreBuilder<T> withLoggingEnabled(Map<String, String> config);
    StoreBuilder<T> withLoggingDisabled()
    T build();
    Map<String, String> logConfig();
    boolean loggingEnabled();
}


Code Block
languagejava
titleStoreSupplier
public interface StoreSupplier<T extends StateStore> {

    /**
     * Return the name of this state store supplier.
     * This must be a valid Kafka topic name; valid characters are ASCII alphanumerics, '.', '_' and '-'
     *
     * @return the name of this state store supplier
     */
    String name();

    /**
     * Return a new {@link StateStore} instance.
     *
     * @return a new {@link StateStore} instance of type T
     */
    T get();
 
	/**
     * Return a String that is used as the scope for metrics recorded by Metered stores
 	 * @return metricsScope
 	 */
    String metricsScope();

}


Code Block
languagejava
titleWindowBytesStoreSupplier
/**
 * A store supplier that can be used to create one or more {@link WindowStore} instances of type &lt;Byte, byte[]&gt;
 */
public interface WindowBytesStoreSupplier extends StoreSupplier<WindowStore<Bytes, byte[]>> {
    /**
     * The number of segments the store has. If your store is segmented then this should be the number of segments
     * in the underlying store. It is also used to reduce the amount of data that is scanned when caching is enabled
     *
     * @return number of segments
     */
    int segments();

    /**
     * The size of the windows any store created from this supplier is creating
     * @return window size
     */
    long windowSize();

    /**
     * Whether or not this store is retaining duplicate keys. Usually only true if the store is being used
     * for joins. Note this should return false if caching is enabled
     * @return true if duplicates should be retained
     */
    boolean retainDuplicates();

    /**
     * The time period for which the {@link WindowStore} will retain historic data
     * @return retentionPeriod
     */
    long retentionPeriod();
}


Code Block
languagejava
titleKeyValueBytesStoreSupplier
/**
 * A store supplier that can be used to create one or more {@link KeyValueStore} instances of type &lt;Byte, byte[]&gt;
 */
public interface KeyValueBytesStoreSupplier extends StoreSupplier<KeyValueStore<Bytes, byte[]>> {

}


Code Block
/**
 * A store supplier that can be used to create one or more {@link SessionStore} instances of type &lt;Byte, byte[]&gt;
 */
public interface SessionBytesStoreSupplier extends StoreSupplier<SessionStore<Bytes, byte[]>> {

    /**
     * The size of a segment, in milliseconds. Used when caching is enabled to segment the cache
     * and reduce the amount of data that needs to be scanned when performing range queries
     *
     * @return segmentInterval in milliseconds
     */
    long segmentIntervalMs();
}

 

Proposed Changes

Add the above methods, interfaces, classes to the DSL. Deprecate existing overloads on KStream, KTable, and KGroupedStream that take more than the required parameters, for example, KTable#filter(Predicate, String) and KTable#filter(Predicate, StateStoreSupplier) will be deprecated. StateStoreSupplier will also be deprecated. All versions of KTable#through and KTable#to will be deprecated in favour of using KTable#toStream()#through and  KTable#toStream()#to  

The new Interface BytesStoreSupplier supersedes the existing StateStoreSupplier (which will remain untouched). This so we can provide a convenient way for users creating custom state stores to wrap them with caching/logging etc if they chose. In order to do this we need to force the inner most store, i.e, the custom store, to be a store of type `<Bytes, byte[]>`. Add the above methods, interfaces, classes to the DSL. Deprecate the existing overloads.

Compatibility, Deprecation, and Migration Plan

...