Versions Compared

Key

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

...

The StateStoreSupplier argument stands out as a raw type. Making it type-parameterised will help detect prior to runtime errors such as when someone refactors their app to use a different typo of aggregations (e.g. TimeWindowed vs SessionWindowed) and forgets to change the StateStoreSupplier passed in.

Public Interfaces

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

...

Binary log format

...

The network protocol and api behavior

...

Any class in the public packages under clientsConfiguration, especially client configuration

  • org/apache/kafka/common/serialization

  • org/apache/kafka/common

  • org/apache/kafka/common/errors

  • org/apache/kafka/clients/producer

  • org/apache/kafka/clients/consumer (eventually, once stable)

...

Monitoring

...

Command line tools and arguments

  • KGroupedStream:

    • - KTable<K, Long> count(final StateStoreSupplier<KeyValueStore> storeSupplier);
      + 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> storeSupplier);
      + final StateStoreSupplier<WindowStore<K,Long>> storeSupplier);

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

    • KTable<K, V> reduce(final Reducer<V> reducer,
      - final StateStoreSupplier<KeyValueStore> storeSupplier);
      + 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> storeSupplier);
      + final StateStoreSupplier<WindowStore<K,V>> storeSupplier);

    • KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
      final SessionWindows sessionWindows,
      - final StateStoreSupplier<SessionStore> storeSupplier);
      + 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> storeSupplier);
      + 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> storeSupplier);
      + final StateStoreSupplier<WindowStore<K,VR>> storeSupplier);

  • KGroupedTable
    • - KTable<K, Long> count(final StateStoreSupplier<KeyValueStore> storeSupplier);
      + 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> storeSupplier);
      + 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> storeSupplier);
      + final StateStoreSupplier<KeyValueStore<K,VR>> storeSupplier);

  • Stores
    • 2 new interfaces:
    • public interfaces:

      • PersistentWindowFactory:

         PersistentWindowFactory<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
        + */
        + PersistentWindowFactory<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
        + */
        + PersistentWindowFactory<K, V> disableLogging();
        +
        + /**
        + * Caching should be enabled on the created store.
        + * @return the factory to create a persistent key-value store
        + */
        + PersistentWindowFactory<K, V> enableCaching();
        + /**
        + * Return the instance of StateStoreSupplier of new key-value store.
        + * @return the key-value store; never null
        + */
        + StateStoreSupplier<WindowStore<K, V>> build();
        +
        + }
        PersistentSessionFactory:

        + 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<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, V> sessionWindowed(final long retentionPeriod);

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

      • changes to InMemoryKeyValueFactory:
        • - StateStoreSupplier build();
          + 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.

...