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:
      • PersistentWindowFactory:

         

        Code Block
        public interface PersistentWindowFactory<K, V> {
        
        
        /**
        * Caching should be enabled on the created store.
        * @return the factory to create a persistent key-value 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 key-value 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 key-value store
        */
        PersistentWindowFactory<K, V> enableLogging(final Map<String, String> config);
        
        
        /**
        * Return the instance of StateStoreSupplier of new key-value store.
        * @return the key-value store; never null
        */
        StateStoreSupplier<WindowStore<K, V>> build();
        }
      • PersistentSessionFactory:
        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 key-value store

        +
        
        */

        +
        
        PersistentSessionFactory<K, V> disableLogging();

        +
        +
        
        /**

        +
        
        * Caching should be enabled on the created store.

        +
        
        * @return the factory to create a persistent key-value store

        +
        
        */

        +
        
        PersistentSessionFactory<K, V> enableCaching();

        +
        
        /**

        +
        
        * Return the instance of StateStoreSupplier of new key-value store.

        +
        
        * @return the key-value store; never null

        +
        
        */

        +
        
        StateStoreSupplier<SessionStore<K, V>> build();
         
        }
      • changes to PersistentKeyValueFactory :
        • - PersistentKeyValueFactory<KPersistentWindowFactory<K, V> windowed(final long windowSize, long retentionPeriod, int numSegments, boolean retainDuplicates);+ PersistentWindowFactory<K, V> windowed(final long windowSize, long retentionPeriod, int numSegments, boolean retainDuplicates);- PersistentKeyValueFactory<K, V> sessionWindowed(final long retentionPeriod);
          + PersistentSessionFactory<K

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

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

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

 

Proposed Changes

...

Compatibility, Deprecation, and Migration Plan

...

The impact on existing users should be minimal. 

Rejected Alternatives

If 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.

...