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 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 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 VV> predicate, 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[]>> inMemoryKeyValueStore(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,
                                                           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 WindowBytesStoreSupplier supplier, 
																	  final Serde<K> keySerde, 
																      final Serde<V> valueSerde)

public <K, V> StateStoreBuilder<KeyValueStore<K, V>> keyValueStoreBuilder(final KeyValueBytesStoreSupplier   supplier,
 																		  final SessionWindowsSerde<K> windows) 

/**
 *  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
 */ 
keySerde, 
																          final Serde<V> valueSerde)

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

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


Code Block
languagejava
titleTopology
public synchronized <K, V> Topology addGlobalStore(final StateStoreBuilder storeSupplier,
                final Serde<V> valueSerde)

public <K, V> StateStoreBuilder<SessionStore<K, V>> sessionStoreBuilder(final BytesStoreSupplier<SessionStore<Bytes, byte[]>> supplier,
																	    final Serde<K> keySerde, 
																        final Serde<V> valueSerde)
Code Block
languagejava
titleTopology
public synchronized <K, V> Topology addGlobalStore(final StateStoreBuilder storeSupplier,
        final String sourceName,
                                                final String sourceName,
                                                          final TimestampExtractor timestampExtractor,
                                                          final Deserializer keyDeserializer,
                                                          final Deserializer valueDeserializer,
                                                          final String topic,
                                                          final String processorName,
                                                          final ProcessorSupplier stateUpdateSupplier)
 
public synchronized final Topology 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> windowedBy(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,                            final Materialized<K, VR, KeyValueStore<K, VR>> materialized);byte[]>> materialized);


Code Block
Code Block
languagejava
titleKGroupedTable
KTable<K, Long> count(final Materialized<K, V, KeyValueStore<KKeyValueStore<Bytes, V>>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
titleStreamsBuilder
public synchronized <K, V> KStream<K, V> stream(final Collection<String>String topic, final Consumed<K, V> options)
 
public synchronized <K, V> KStream<K, V> stream(final PatternString patterntopic, final Consumed<K, V> options)
 
public synchronized <K, V> KTable<KKStream<K, V> tablestream(final StringCollection<String> topic, final Consumed<K, V> consumedoptions)
 
public synchronized <K, V> KTable<KKStream<K, V> tablestream(final StringCollection<String> topic)

public synchronized <K, finalV> Consumed<KKStream<K, V> consumedstream(final Pattern pattern, final Materialized<KConsumed<K, V> materializedoptions)

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

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

 

New classes and interfaces:

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

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

    KTable<Windowed<K>, Long> count(table(final String topic, final Materialized<K, LongV, WindowStore<KKeyValueStore<Bytes, Long>> materializedAs);byte[]>> materialized)

public synchronized <K, V> <VR> KTable<Windowed<K>GlobalKTable<K, VR>V> aggregateglobalTable(final Initializer<VR>String initializertopic,
 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> {

   final Aggregator<?KTable<Windowed<K>, super K, ? super V, VR> aggregatorLong> count();

    KTable<Windowed<K>, Long> count(final Materialized<K, Long, WindowStore<Bytes, byte[]>> 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 Materialized<K, VR, WindowStore<K, VR>> materializedAs);


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

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


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

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

    KTable<Windowed<K>, Long>V> countreduce(final Materialized<K,Reducer<V> Longreducer,
 SessionStore<K, Long>> materializedAs);

    <VR, T> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
                     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 Aggregator<? super K, ? super V, VR> aggregatorMaterialized<K, Long, SessionStore<Bytes, byte[]>> materializedAs);

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

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

    <VR, T> KTable<Windowed<K>, VR> aggregator aggregate(final Initializer<VR> initializer,
                                              final Merger<Aggregator<? super K, T> ? super V, VR> aggregator,
                                              final Merger<? super K, T> sessionMerger,
                                              final Materialized<K, VR, SessionStore<KSessionStore<Bytes, VR>>byte[]>> materializedAs);


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

    KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                  final Materialized<K, V, SessionStore<KSessionStore<Bytes, V>>byte[]>> materializedAs);
}


Code Block
languagejava
titleMaterialized
/**
 * Used when materializing a state store, i.e, during an aggregation operation or KTable operations
 */
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> V> Materialized<K, V, WindowStore<Bytes, S>byte[]>> as(final BytesStoreSupplier<S>WindowBytesStoreSupplier supplier);
 
    public static <K, V> Materialized<K, V, S> withValueSerdeSessionStore<Bytes, byte[]>> as(final Serde<V>SessionBytesStoreSupplier valueSerdesupplier);
 
    public static <K, V> Materialized<K, V, S> withKeySerde(KeyValueStore<Bytes, byte[]>> as(final Serde<K>KeyValueBytesStoreSupplier valueSerdesupplier)
 
    public Materialized<K, V, S> withLoggingEnabledwithValueSerde(final Map<String, String> topicConfigSerde<V> valueSerde)

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

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

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

Code Block
languagejava
titleSerialized
/**
 * Optional params thatpublic canMaterialized<K, beV, passed to groupBy and groupByKey operations
 */
public class Serialized<K, V> {

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

    public Serialized<K, V> withKeySerde(final Serde<K> keySerde)

    public Serialized<K, V> withValueSerde(final Serde valueSerde)
}S> withCachingEnabled()

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

	public String storeName();

	public StoreSupplier<S> storeSupplier()

	public Serde<K> keySerde()

	public Serde<V> valueSerde()

	public boolean loggingEnabled()

	public Map<String, String> logConfig()

	public boolean cachingEnabled()

} 


Code Block
languagejava
titleJoinedSerialized
/**
 * Optional params that can be passed to join,groupBy leftJoin,and outerJoingroupByKey operations
 */
public class Joined<KSerialized<K, V,V> VO> {

    public static <K, V, VO> Joined<KV> Serialized<K, V,V> VO> with(final Serde<K> keySerde, final SerdeSerde<V> <V> valueSerde, final Serde<VO> otherValueSerde)

    public static <KSerialized<K, V, VO> Joined<K, V, VO> keySerdeV> withKeySerde(final Serde<K> keySerde)

    public static <KSerialized<K, V, VO> Joined<K, V, VO> valueSerde(finalV> withValueSerde(final Serde valueSerde)
 
	public Serde<K> keySerde()
 
    public Serde<V> valueSerde()
}


Code Block
languagejava
titleJoined
/**
 * Optional  public static <K, V, VO>params that can be passed to join, leftJoin, outerJoin operations
 */
public class Joined<K, V, VO> otherValueSerde(final Serde<V> valueSerde){

    public static Joined<K<K, V, VO> withKeySerdeJoined<K, V, VO> with(final Serde<K> keySerde, final Serde <V> valueSerde, final Serde<VO> otherValueSerde)

    public static <K, V, VO> Joined<K, V, VO> withValueSerdekeySerde(final Serde<V>Serde<K> valueSerdekeySerde)

    public static <K, V, VO> Joined<K, V, VO> withOtherValueSerdevalueSerde(final Serde<VO>Serde<V> otherValueSerdevalueSerde)
}
Code Block
languagejava
titleProduced
/**
 * Optional arguments thatpublic canstatic be specified when doing to and through operations
 */
public static <K, V> Produced<K, V> with<K, V, VO> Joined<K, V, VO> otherValueSerde(final Serde<V> valueSerde)

    public Joined<K, V, VO> withKeySerde(final Serde<K> keySerde)

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

    public static <KJoined<K, V> Produced<KV, V>VO> withwithOtherValueSerde(final Serde<VO> otherValueSerde)
 
	public Serde<K> keySerde, final()
 
    public Serde<V> valueSerde()
 
	public Serde<VO> otherValueSerde()
}


Code Block
languagejava
titleProduced
/**
 * Optional arguments that can be specified when doing to and through operations
 */, final StreamPartitioner<K, V> partitioner) 

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

public static <K, V> Produced<K, V> valueSerdewith(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
 
/**
 * 
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
/**
 * 
Code Block
languagejava
titleStateStoreBuilder
/**
 * Implementations of this will provide the ability to wrap a given StateStore
 * with or without caching/loggging etc.
 */
public interface StateStoreBuilder<TStoreBuilder<T extends StateStore> {

    StateStoreBuilder<T>StoreBuilder<T> withCachingEnabled();
    StateStoreBuilder<T>StoreBuilder<T> withLoggingEnabled(Map<String, String> config);
    T build();
};
    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[]>> {

    /*
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 '-'
     *
     * @returnThe thesize name of thisa statesegment, storein supplier
milliseconds. Used when caching is */
enabled to segment  String name();

the cache
     /**
 and reduce the amount *of Returndata athat newneeds {@linkto StateStore} instance.
     *be scanned when performing range queries
     *
 @return a new {@link StateStore}* instance@return ofsegmentInterval typein Tmilliseconds
     */
    Tlong getsegmentIntervalMs();

}

 

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 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[]>`. 

...