...
KGroupedStream -deprecate the following methods:
KTable<K, Long> count(final StateStoreSupplier<KeyValueStore> storeSupplier);
<W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
final StateStoreSupplier<WindowStore> storeSupplier);KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows,
final StateStoreSupplier<SessionStore> storeSupplier);KTable<K, V> reduce(final Reducer<V> reducer,
final StateStoreSupplier<KeyValueStore> storeSupplier);<W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final Windows<W> windows,
final StateStoreSupplier<WindowStore> storeSupplier);KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final SessionWindows sessionWindows,final StateStoreSupplier<SessionStore> storeSupplier);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> aggregator,final StateStoreSupplier<KeyValueStore> storeSupplier);
<W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> aggregator,
final Windows<W> windows,
final StateStoreSupplier<WindowStore> storeSupplier);<T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer,
final Aggregator<? super K, ? super V, T> aggregator,
final Merger<? super K, T> sessionMerger,
final SessionWindows sessionWindows,
final Serde<T> aggValueSerde,
final StateStoreSupplier<SessionStore> storeSupplier);
KGroupedStream - add the following replacement methods:
KTable<K, Long> count(final TypedStateStoreSupplier<KeyValueStore<K, Long>> storeSupplier);
<W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
final TypedStateStoreSupplier<WindowStore<K, Long>> storeSupplier);KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows,
final TypedStateStoreSupplier<SessionStore<K, Long>> storeSupplier);KTable<K, V> reduce(final Reducer<V> reducer,
final TypedStateStoreSupplier<KeyValueStore<K ,V>> storeSupplier);<W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final Windows<W> windows,
final TypedStateStoreSupplier<WindowStore<K, V>> storeSupplier);KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final SessionWindows sessionWindows,final TypedStateStoreSupplier<SessionStore<K, V>> storeSupplier);
<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> aggregator,final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);
<W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> aggregator,
final Windows<W> windows,
final TypedStateStoreSupplier<WindowStore<K, VR>> storeSupplier);<T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer,
final Aggregator<? super K, ? super V, T> aggregator,
final Merger<? super K, T> sessionMerger,
final SessionWindows sessionWindows,
final Serde<T> aggValueSerde,
final TypedStateStoreSupplier<SessionStore<K, T>> storeSupplier);
- KGroupedTable - deprecate the following methods:
KTable<K, Long> count(final StateStoreSupplier<KeyValueStore> storeSupplier);
KTable<K, V> reduce(final Reducer<V> adder,
final Reducer<V> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> adder,
final Aggregator<? super K, ? super V, VR> subtractor,
final StateStoreSupplier<KeyValueStore> storeSupplier);
- KGroupedTable - add the following replacement methods:
KTable<K, Long> count(final TypedStateStoreSupplier<KeyValueStore<K, Long>> storeSupplier);
KTable<K, V> reduce(final Reducer<V> adder,
final Reducer<V> subtractor,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);<VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
final Aggregator<? super K, ? super V, VR> adder,
final Aggregator<? super K, ? super V, VR> subtractor,
final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);
- KTable - deprecate the following methods:
- KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
- KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
final Serde<VR> valueSerde,
final StateStoreSupplier<KeyValueStore> storeSupplier);KTable<K, V> through(final String topic,
final StateStoreSupplier<KeyValueStore> storeSupplier);KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
final String topic,
final StateStoreSupplier<KeyValueStore> storeSupplier);KTable<K, V> through(final Serde<K> keySerde, Serde<V> valSerde,
final String topic,
final StateStoreSupplier<KeyValueStore> storeSupplier);KTable<K, V> through(final Serde<K> keySerde,
final Serde<V> valSerde,
final StreamPartitioner<? super K, ? super V> partitioner,
final String topic,
final StateStoreSupplier<KeyValueStore> storeSupplier);<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final StateStoreSupplier<KeyValueStore> storeSupplier);
- KTable - add the following replacement methods:
- KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);
- KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
final Serde<VR> valueSerde,
final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);KTable<K, V> through(final String topic,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
final String topic,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);KTable<K, V> through(final Serde<K> keySerde, Serde<V> valSerde,
final String topic,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);KTable<K, V> through(final Serde<K> keySerde,
final Serde<V> valSerde,
final StreamPartitioner<? super K, ? super V> partitioner,
final String topic,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier);<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);<VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);<VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final TypedStateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);
- KStreamBuilder - deprecate the following method:
<K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
final Serde<V> valSerde,
final String topic,
final StateStoreSupplier<KeyValueStore> storeSupplier)
- KStreamBuilder - add the following replacement method:
- <K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
final Serde<V> valSerde,
final String topic,
final TypedStateStoreSupplier<KeyValueStore<K, V>> storeSupplier)
- <K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
- Deprecate Stores class
- 2 new Add a new replacement TypedStores class with stronger typing of public interfaces:
Code Block public interface PersistentWindowFactory<K, V> { /** * Caching should be enabled on the created store. * @return the factory to create a persistent window store */ PersistentWindowFactory<K, V> enableCaching(); /** * Indicates that a changelog should not be created for the key-value store * @return the factory to create a persistent window store */ PersistentWindowFactory<K, V> disableLogging(); /** * Indicates that a changelog should be created for the store. The changelog will be created * with the provided cleanupPolicy and configs. * * Note: Any unrecognized configs will be ignored. * @param config any configs that should be applied to the changelog * @return the factory to create a persistent window store */ PersistentWindowFactory<K, V> enableLogging(final Map<String, String> config); /** * Return the instance of StateStoreSupplier of new window store. * @return the key-value store; never null */
StateStoreSupplier<WindowStore<KTypedStateStoreSupplier<WindowStore<K, V>> build(); }
Code Block public interface PersistentSessionFactory<K, V> { /** * Indicates that a changelog should be created for the store. The changelog will be created * with the provided cleanupPolicy and configs. * * Note: Any unrecognized configs will be ignored. * @param config any configs that should be applied to the changelog * @return the factory to create a persistent key-value store */ PersistentSessionFactory<K, V> enableLogging(final Map<String, String> config); /** * Indicates that a changelog should not be created for the key-value store * @return the factory to create a persistent session store */ PersistentSessionFactory<K, V> disableLogging(); /** * Caching should be enabled on the created store. * @return the factory to create a persistent session store */ PersistentSessionFactory<K, V> enableCaching(); /** * Return the instance of StateStoreSupplier of new session store. * @return the key-value store; never null */
StateStoreSupplier<SessionStore<KTypedStateStoreSupplier<SessionStore<K, V>> build(); }
changes to PersistentKeyValueFactory - differences in PersistentKeyValueFactory in TypedStores versus that in Stores:
PersistentWindowFactory<K, V> windowed(final long windowSize, long retentionPeriod, int numSegments, boolean retainDuplicates);
PersistentSessionFactory<K, V> sessionWindowed(final long retentionPeriod);
TypedStateStoreSupplier<KeyValueStore<K, V>> build();
changes to InMemoryKeyValueFactory- differences in PersistentKeyValueFactory in TypedStores versus that in Stores:
- differences in InMemoryKeyValueFactory in TypedStores versus that in Stores:
TypedStateStoreSupplier<KeyValueStore<K, V>> build();
- differences in InMemoryKeyValueFactory in TypedStores versus that in Stores:
Proposed Changes
The new usage would be e.g.:
...
Add type parameters to current method parameters. This has been rejected as a backwards-incompatible change.
<K, V>