You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 2 Next »

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state"Under Discussion"

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

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

Motivation

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

StateStoreSupplier countStore = Stores.create("Counts")
    .withKeys(Serdes.String())
    .withValues(Serdes.Long())
    .persistent()
    .build();

However, StateStoreSupplier is a generic interface that take the store type as a parameter:

public interface StateStoreSupplier<T extends StateStore>

In the above example that type parameter is lost as the build() method returns a raw type.

As StateStoreSupplier is passed to count/reduce/aggregate etc. methods on KGroupedStream or KGroupedTable, the compiler cannot detect that a supplier for the wrong kind of store is provided.

The other parameters to those methods, such as Serdes, Reducers, etc are type-parameterised by the key and value types allowing compile-time type checks.

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

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

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

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.

  • No labels