Versions Compared

Key

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

Table of Contents

Status

Current state:  "Under Discussion" Discarded (covered by KIP-182)

Discussion thread: here

JIRA: TBD

PRhere

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

 

Note: This KIP may become obsolete as the discussion on "Streams DSL/StateStore Refactoring" supersedes it.

Motivation

The recommended practice to create a StateStoreSupplier is as per the following example:

...

Public Interfaces

  • KGroupedStream -deprecate the following methods:

    • KTable<K, Long> count(final StateStoreSupplier<KeyValueStore<K, Long>> StateStoreSupplier<KeyValueStore> storeSupplier);

    • <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
                                                                                                     final StateStoreSupplier<WindowStore<K, Long>> StateStoreSupplier<WindowStore> storeSupplier);

    • KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows,
                                                                  final StateStoreSupplier<SessionStore<K, Long>> StateStoreSupplier<SessionStore> storeSupplier);

    • KTable<K, V> reduce(final Reducer<V> reducer,
                                         final StateStoreSupplier<KeyValueStore<K ,V>> StateStoreSupplier<KeyValueStore> storeSupplier);

    • <W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                                                                                  final Windows<W> windows,
                                                                                                  final StateStoreSupplier<WindowStore<K, V>> StateStoreSupplier<WindowStore> storeSupplier);

    • KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                                              final SessionWindows sessionWindows,

                                                              final StateStoreSupplier<SessionStore<K, V>> StateStoreSupplier<SessionStore> storeSupplier);

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

                                                           final StateStoreSupplier<KeyValueStore<K, VR>> 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<K, VR>> 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<K, T>> StateStoreSupplier<SessionStore> storeSupplier);

    KGroupedTable
  • KGroupedStream - add the following replacement methods:

    • KTable<K, Long> count(final TypedStateStoreSupplier<KeyValueStore<K, Long>> storeSupplier);

       

    • KTable<K, V> reduce(final Reducer<V> adder,
      <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
                                                                 final Reducer<V> subtractor,
                                           final StateStoreSupplier<KeyValueStore TypedStateStoreSupplier<WindowStore<K, V>Long>> storeSupplier);

       

    • <VR> KTable<K, VR> aggregate(final Initializer<VR> initializerKTable<Windowed<K>, Long> count(final SessionWindows sessionWindows,
                                                           final Aggregator<? super K, ? super V, VR> adder,
                      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 Aggregator<? super K, ? super V, VR> subtractor,
                                                           final StateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);                                      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)
  • Deprecate Stores class
  • Add a new replacement TypedStores class with the following extra public interfaces in addition to those equivalent to those in Stores class:
      • Code Block
        public interface PersistentWindowFactory<K, V> {
        
        
        /**
        * Caching should be enabled on the created store.
    Stores
      2 new public interfaces: Code Blockpublic 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
      • enableCaching();
      • 
        
        /**
        *
    • Return the instance of StateStoreSupplier of new window store.
      •  Indicates that a changelog should not be created for the key-value store
        * @return the
    • key-value store; never null
      •  factory to create a persistent window store
        */
        
    • StateStoreSupplier<WindowStore<K
      • PersistentWindowFactory<K, 
    • V>>
      • V> 
    • build
      • disableLogging();
        
    • } Code Blockpublic 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
      •  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 
    • factory to create a persistent key-value store
      • key-value store; never null
        */
        
    • PersistentSessionFactory<K
      • TypedStateStoreSupplier<WindowStore<K, 
    • V>
      • V>> 
    • enableLogging(final Map<String, String> config);
      • build();
        }
      • Code Block
        public interface PersistentSessionFactory<K, V> {
        /**
        * 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
      •  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 
    • enabled
      • applied 
    • on
      • to the 
    • created store.
      • changelog
        * @return the factory to create a persistent 
    • session
      • key-value store
        */
        PersistentSessionFactory<K, V>
    • enableCaching(
      •  enableLogging(final Map<String, String> config);
        /**
        * 
    • Return
      • Indicates 
    • the
      • that 
    • instance
      • a 
    • of
      • changelog 
    • StateStoreSupplier
      • should 
    • of
      • not 
    • new
      • be 
    • session store. * @return
      • created for the key-value store
    • ;
      • 
        * 
    • never null */ StateStoreSupplier<SessionStore<K, V>> build();   }
    • changes to PersistentKeyValueFactory :
      • PersistentWindowFactory<K, V> windowed(final long windowSize, long retentionPeriod, int numSegments, boolean retainDuplicates);

      • PersistentSessionFactory<K, V> sessionWindowed(final long retentionPeriod);

      • StateStoreSupplier<KeyValueStore<K, V>> build();

    • changes to InMemoryKeyValueFactory:
      • StateStoreSupplier<KeyValueStore<K, V>> build();

 

Proposed Changes

The new usage would be e.g.:

Code Block
StateStoreSupplier<KeyValueStore<String, Long>> countStore = Stores.create("Counts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
    .build();
Code Block
StateStoreSupplier<WindowStore<String, Long>> windowedStore = Stores.create("WindowedCounts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
	.windowed(1000, 10000, 10, false)
    .build();
      • @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
        */
        TypedStateStoreSupplier<SessionStore<K, V>> build();
         
        }
      • 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();

      • differences in InMemoryKeyValueFactory in TypedStores versus that in Stores:
        • TypedStateStoreSupplier<KeyValueStore<K, V>> build();

 

Proposed Changes

Pull Request to demonstrate the changes: https://github.com/apache/kafka/pull/2992/files

 

The new usage would be e.g.:

Code Block
TypedStateStoreSupplier<KeyValueStore<String, Long>> countStore = TypedStores.create("Counts
Code Block
StateStoreSupplier<SessionStore<String, Long>> sessionStore = Stores.create("SessionWindowedCounts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
	.sessionWindowed(60000)
    .build();

...

Code Block

Compatibility, Deprecation, and Migration Plan

 

These changes are, in general, not backward compatible in that some mechanical but manual changes to user code may be required.

Most case including usage as per the documentation (see examples in preceding sections) no changes would be required, although users should be encouraged to add type parameters to the previously used raw types to get rid of compilation warnings or to remove @SupressWarnings annotations.

However, some mechanical changes will be required if StateStoreSupplier type parameterised with the raw SessionStore type was used in a cast, for instance. See: KGroupedStreamImplTest.java#L98

Another case where changes are needed would be if results of PersistentKeyValueFactory .sessionWindowed() or .windowed() or results of calls to enableLogging(), disableLogging(), enableCaching() on return values of .sessionWindowed() or .windowed() were assigned to a variable, field or used as a parameter. In that case the type of the variable, field or parameter would need changing from PersistentKeyValueFactory to PersistentSessionFactory<K, V> or PersistentWindowFactory<K, V> respectively.

TypedStateStoreSupplier<WindowStore<String, Long>> windowedStore = TypedStores.create("WindowedCounts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
	.windowed(1000, 10000, 10, false)
    .build();
Code Block
TypedStateStoreSupplier<SessionStore<String, Long>> sessionStore = TypedStores.create("SessionWindowedCounts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
	.sessionWindowed(60000)
    .build();

 

Compatibility, Deprecation, and Migration Plan

Changes are intended to be backwards-compatibleTo make that straightforward upgrade note shall be provided in the documentation.

Test Plan

Only re-run of existing tests is envisaged at this time.

Rejected Alternatives

Add type parameters to current method parameters. This has been rejected as a backwards-incompatible change.

<K, V>Node considered.