Versions Compared

Key

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

...

  • KGroupedStream

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

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

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

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

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

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

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

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

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

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

       

    • KTable<K, V> reduce(final Reducer<V> adder,
                                         final Reducer<V> subtractor,
                                         final StateStoreSupplier<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 StateStoreSupplier<KeyValueStore<K, VR>> storeSupplier);

  • Stores
    • 2 new 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<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<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

Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.

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();
Code Block
StateStoreSupplier<SessionStore<String, Long>> sessionStore = Stores.create("SessionWindowedCounts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
	.sessionWindowed(60000)
    .build();

 

Compatibility, Deprecation, and Migration Plan

To be evaluated

Test Plan

Describe in few sentences how the KIP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?

Rejected Alternatives

 

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.

To 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

Node consideredIf there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.